diff --git a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java index 79ea0bddfdc..c5716fa6c84 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/rfile/RFileScanner.java @@ -347,10 +347,10 @@ public Iterator> iterator() { for (int i = 0; i < sources.length; i++) { // TODO may have been a bug with multiple files and caching in older version... - FSDataInputStream inputStream = (FSDataInputStream) sources[i].getInputStream(); - CachableBuilder cb = - new CachableBuilder().input(inputStream, "source-" + i).length(sources[i].getLength()) - .conf(opts.in.getConf()).cacheProvider(cacheProvider).cryptoService(cryptoService); + CachableBuilder cb = new CachableBuilder() + .input((FSDataInputStream) sources[i].getInputStream(), "source-" + i) + .length(sources[i].getLength()).conf(opts.in.getConf()).cacheProvider(cacheProvider) + .cryptoService(cryptoService); readers.add(RFile.getReader(cb, sources[i].getRange())); } diff --git a/core/src/main/java/org/apache/accumulo/core/conf/Property.java b/core/src/main/java/org/apache/accumulo/core/conf/Property.java index 53011b7f336..0b4df706b04 100644 --- a/core/src/main/java/org/apache/accumulo/core/conf/Property.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/Property.java @@ -589,11 +589,6 @@ public enum Property { TSERV_COMPACTION_SERVICE_ROOT_PLANNER("tserver.compaction.major.service.root.planner", DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME, "Compaction planner for root tablet service", "2.1.0"), - TSERV_COMPACTION_SERVICE_ROOT_RATE_LIMIT("tserver.compaction.major.service.root.rate.limit", "0B", - PropertyType.BYTES, - "Maximum number of bytes to read or write per second over all major" - + " compactions in this compaction service, or 0B for unlimited.", - "2.1.0"), TSERV_COMPACTION_SERVICE_ROOT_MAX_OPEN( "tserver.compaction.major.service.root.planner.opts.maxOpen", "30", PropertyType.COUNT, "The maximum number of files a compaction will open", "2.1.0"), @@ -606,11 +601,6 @@ public enum Property { TSERV_COMPACTION_SERVICE_META_PLANNER("tserver.compaction.major.service.meta.planner", DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME, "Compaction planner for metadata table", "2.1.0"), - TSERV_COMPACTION_SERVICE_META_RATE_LIMIT("tserver.compaction.major.service.meta.rate.limit", "0B", - PropertyType.BYTES, - "Maximum number of bytes to read or write per second over all major" - + " compactions in this compaction service, or 0B for unlimited.", - "2.1.0"), TSERV_COMPACTION_SERVICE_META_MAX_OPEN( "tserver.compaction.major.service.meta.planner.opts.maxOpen", "30", PropertyType.COUNT, "The maximum number of files a compaction will open", "2.1.0"), @@ -623,11 +613,6 @@ public enum Property { TSERV_COMPACTION_SERVICE_DEFAULT_PLANNER("tserver.compaction.major.service.default.planner", DefaultCompactionPlanner.class.getName(), PropertyType.CLASSNAME, "Planner for default compaction service.", "2.1.0"), - TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT("tserver.compaction.major.service.default.rate.limit", - "0B", PropertyType.BYTES, - "Maximum number of bytes to read or write per second over all major" - + " compactions in this compaction service, or 0B for unlimited.", - "2.1.0"), TSERV_COMPACTION_SERVICE_DEFAULT_MAX_OPEN( "tserver.compaction.major.service.default.planner.opts.maxOpen", "10", PropertyType.COUNT, "The maximum number of files a compaction will open", "2.1.0"), diff --git a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java index 5182c614aa8..6d2df13fd1c 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java @@ -34,7 +34,6 @@ import org.apache.accumulo.core.metadata.TabletFile; import org.apache.accumulo.core.metadata.UnreferencedTabletFile; import org.apache.accumulo.core.spi.crypto.CryptoService; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -171,7 +170,6 @@ protected static class FileOptions { public final TabletFile file; public final FileSystem fs; public final Configuration fsConf; - public final RateLimiter rateLimiter; // writer only objects public final String compression; public final FSDataOutputStream outputStream; @@ -188,15 +186,14 @@ protected static class FileOptions { public final boolean dropCacheBehind; protected FileOptions(AccumuloConfiguration tableConfiguration, TabletFile file, FileSystem fs, - Configuration fsConf, RateLimiter rateLimiter, String compression, - FSDataOutputStream outputStream, boolean enableAccumuloStart, CacheProvider cacheProvider, - Cache fileLenCache, boolean seekToBeginning, CryptoService cryptoService, - Range range, Set columnFamilies, boolean inclusive, boolean dropCacheBehind) { + Configuration fsConf, String compression, FSDataOutputStream outputStream, + boolean enableAccumuloStart, CacheProvider cacheProvider, Cache fileLenCache, + boolean seekToBeginning, CryptoService cryptoService, Range range, + Set columnFamilies, boolean inclusive, boolean dropCacheBehind) { this.tableConfiguration = tableConfiguration; this.file = Objects.requireNonNull(file); this.fs = fs; this.fsConf = fsConf; - this.rateLimiter = rateLimiter; this.compression = compression; this.outputStream = outputStream; this.enableAccumuloStart = enableAccumuloStart; @@ -226,10 +223,6 @@ public Configuration getConfiguration() { return fsConf; } - public RateLimiter getRateLimiter() { - return rateLimiter; - } - public String getCompression() { return compression; } @@ -279,7 +272,6 @@ public static class FileHelper { private TabletFile file; private FileSystem fs; private Configuration fsConf; - private RateLimiter rateLimiter; private CryptoService cryptoService; private boolean dropCacheBehind = false; @@ -303,11 +295,6 @@ protected FileHelper tableConfiguration(AccumuloConfiguration tableConfiguration return this; } - protected FileHelper rateLimiter(RateLimiter rateLimiter) { - this.rateLimiter = rateLimiter; - return this; - } - protected FileHelper cryptoService(CryptoService cs) { this.cryptoService = Objects.requireNonNull(cs); return this; @@ -320,28 +307,27 @@ protected FileHelper dropCacheBehind(boolean drop) { protected FileOptions toWriterBuilderOptions(String compression, FSDataOutputStream outputStream, boolean startEnabled) { - return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, compression, - outputStream, startEnabled, NULL_PROVIDER, null, false, cryptoService, null, null, true, + return new FileOptions(tableConfiguration, file, fs, fsConf, compression, outputStream, + startEnabled, NULL_PROVIDER, null, false, cryptoService, null, null, true, dropCacheBehind); } protected FileOptions toReaderBuilderOptions(CacheProvider cacheProvider, Cache fileLenCache, boolean seekToBeginning) { - return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false, + return new FileOptions(tableConfiguration, file, fs, fsConf, null, null, false, cacheProvider == null ? NULL_PROVIDER : cacheProvider, fileLenCache, seekToBeginning, cryptoService, null, null, true, dropCacheBehind); } protected FileOptions toIndexReaderBuilderOptions(Cache fileLenCache) { - return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false, - NULL_PROVIDER, fileLenCache, false, cryptoService, null, null, true, dropCacheBehind); + return new FileOptions(tableConfiguration, file, fs, fsConf, null, null, false, NULL_PROVIDER, + fileLenCache, false, cryptoService, null, null, true, dropCacheBehind); } protected FileOptions toScanReaderBuilderOptions(Range range, Set columnFamilies, boolean inclusive) { - return new FileOptions(tableConfiguration, file, fs, fsConf, rateLimiter, null, null, false, - NULL_PROVIDER, null, false, cryptoService, range, columnFamilies, inclusive, - dropCacheBehind); + return new FileOptions(tableConfiguration, file, fs, fsConf, null, null, false, NULL_PROVIDER, + null, false, cryptoService, range, columnFamilies, inclusive, dropCacheBehind); } protected AccumuloConfiguration getTableConfiguration() { @@ -388,11 +374,6 @@ public WriterBuilder withCompression(String compression) { return this; } - public WriterBuilder withRateLimiter(RateLimiter rateLimiter) { - rateLimiter(rateLimiter); - return this; - } - public WriterBuilder dropCachesBehind() { this.dropCacheBehind(true); return this; @@ -441,11 +422,6 @@ public ReaderBuilder withFileLenCache(Cache fileLenCache) { return this; } - public ReaderBuilder withRateLimiter(RateLimiter rateLimiter) { - rateLimiter(rateLimiter); - return this; - } - public ReaderBuilder dropCachesBehind() { this.dropCacheBehind(true); return this; diff --git a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java index 0beb67f535f..8bedcef04e4 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java +++ b/core/src/main/java/org/apache/accumulo/core/file/blockfile/impl/CachableBlockFile.java @@ -33,17 +33,14 @@ import org.apache.accumulo.core.file.rfile.bcfile.BCFile; import org.apache.accumulo.core.file.rfile.bcfile.BCFile.Reader.BlockReader; import org.apache.accumulo.core.file.rfile.bcfile.MetaBlockDoesNotExist; -import org.apache.accumulo.core.file.streams.RateLimitedInputStream; import org.apache.accumulo.core.spi.cache.BlockCache; import org.apache.accumulo.core.spi.cache.BlockCache.Loader; import org.apache.accumulo.core.spi.cache.CacheEntry; import org.apache.accumulo.core.spi.crypto.CryptoService; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.Seekable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,11 +66,10 @@ public static String pathToCacheId(Path p) { public static class CachableBuilder { String cacheId = null; - IoeSupplier inputSupplier = null; + IoeSupplier inputSupplier = null; IoeSupplier lengthSupplier = null; Cache fileLenCache = null; volatile CacheProvider cacheProvider = CacheProvider.NULL_PROVIDER; - RateLimiter readLimiter = null; Configuration hadoopConf = null; CryptoService cryptoService = null; @@ -109,7 +105,7 @@ public CachableBuilder fsPath(FileSystem fs, Path dataFile, boolean dropCacheBeh return this; } - public CachableBuilder input(InputStream is, String cacheId) { + public CachableBuilder input(FSDataInputStream is, String cacheId) { this.cacheId = cacheId; this.inputSupplier = () -> is; return this; @@ -130,11 +126,6 @@ public CachableBuilder cacheProvider(CacheProvider cacheProvider) { return this; } - public CachableBuilder readLimiter(RateLimiter readLimiter) { - this.readLimiter = readLimiter; - return this; - } - public CachableBuilder cryptoService(CryptoService cryptoService) { this.cryptoService = cryptoService; return this; @@ -145,7 +136,6 @@ public CachableBuilder cryptoService(CryptoService cryptoService) { * Class wraps the BCFile reader. */ public static class Reader implements Closeable { - private final RateLimiter readLimiter; // private BCFile.Reader _bc; private final String cacheId; private CacheProvider cacheProvider; @@ -155,7 +145,7 @@ public static class Reader implements Closeable { private final Configuration conf; private final CryptoService cryptoService; - private final IoeSupplier inputSupplier; + private final IoeSupplier inputSupplier; private final IoeSupplier lengthSupplier; private final AtomicReference bcfr = new AtomicReference<>(); @@ -185,8 +175,7 @@ private BCFile.Reader getBCFile(byte[] serializedMetadata) throws IOException { BCFile.Reader reader = bcfr.get(); if (reader == null) { - RateLimitedInputStream fsIn = - new RateLimitedInputStream((InputStream & Seekable) inputSupplier.get(), readLimiter); + FSDataInputStream fsIn = inputSupplier.get(); BCFile.Reader tmpReader = null; if (serializedMetadata == null) { if (fileLenCache == null) { @@ -385,7 +374,6 @@ public Reader(CachableBuilder b) { this.lengthSupplier = b.lengthSupplier; this.fileLenCache = b.fileLenCache; this.cacheProvider = b.cacheProvider; - this.readLimiter = b.readLimiter; this.conf = b.hadoopConf; this.cryptoService = Objects.requireNonNull(b.cryptoService); } diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java index cf3e9f64e2b..46be67bf1e1 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/RFileOperations.java @@ -58,8 +58,7 @@ private static RFileSKVIterator getReader(FileOptions options) throws IOExceptio CachableBuilder cb = new CachableBuilder() .fsPath(options.getFileSystem(), options.getFile().getPath(), options.dropCacheBehind) .conf(options.getConfiguration()).fileLen(options.getFileLenCache()) - .cacheProvider(options.cacheProvider).readLimiter(options.getRateLimiter()) - .cryptoService(options.getCryptoService()); + .cacheProvider(options.cacheProvider).cryptoService(options.getCryptoService()); return RFile.getReader(cb, options.getFile()); } @@ -156,8 +155,7 @@ protected FileSKVWriter openWriter(FileOptions options) throws IOException { } } - BCFile.Writer _cbw = new BCFile.Writer(outputStream, options.getRateLimiter(), compression, - conf, options.cryptoService); + BCFile.Writer _cbw = new BCFile.Writer(outputStream, compression, conf, options.cryptoService); return new RFile.Writer(_cbw, (int) blockSize, (int) indexBlockSize, samplerConfig, sampler); } diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java index e3adf9e5175..928b252b168 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/SplitLarge.java @@ -92,10 +92,9 @@ public void execute(String[] args) throws Exception { int blockSize = (int) aconf.getAsBytes(Property.TABLE_FILE_BLOCK_SIZE); try ( Writer small = new RFile.Writer( - new BCFile.Writer(fs.create(new Path(smallName)), null, "gz", conf, cs), blockSize); + new BCFile.Writer(fs.create(new Path(smallName)), "gz", conf, cs), blockSize); Writer large = new RFile.Writer( - new BCFile.Writer(fs.create(new Path(largeName)), null, "gz", conf, cs), - blockSize)) { + new BCFile.Writer(fs.create(new Path(largeName)), "gz", conf, cs), blockSize)) { small.startDefaultLocalityGroup(); large.startDefaultLocalityGroup(); diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java index 9b5683ce222..890dc58ab5d 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/BCFile.java @@ -38,7 +38,6 @@ import org.apache.accumulo.core.crypto.CryptoUtils; import org.apache.accumulo.core.file.rfile.bcfile.Utils.Version; import org.apache.accumulo.core.file.streams.BoundedRangeFileInputStream; -import org.apache.accumulo.core.file.streams.RateLimitedOutputStream; import org.apache.accumulo.core.file.streams.SeekableDataInputStream; import org.apache.accumulo.core.spi.crypto.CryptoEnvironment; import org.apache.accumulo.core.spi.crypto.CryptoEnvironment.Scope; @@ -47,7 +46,6 @@ import org.apache.accumulo.core.spi.crypto.FileEncrypter; import org.apache.accumulo.core.spi.crypto.NoFileDecrypter; import org.apache.accumulo.core.spi.crypto.NoFileEncrypter; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -104,7 +102,7 @@ private BCFile() { * BCFile writer, the entry point for creating a new BCFile. */ public static class Writer implements Closeable { - private final RateLimitedOutputStream out; + private final FSDataOutputStream out; private final Configuration conf; private FileEncrypter encrypter; private CryptoEnvironmentImpl cryptoEnvironment; @@ -131,18 +129,18 @@ private static final class WBlockState { private final CompressionAlgorithm compressAlgo; private Compressor compressor; // !null only if using native // Hadoop compression - private final RateLimitedOutputStream fsOut; + private final FSDataOutputStream fsOut; private final OutputStream cipherOut; private final long posStart; private final SimpleBufferedOutputStream fsBufferedOutput; private OutputStream out; - public WBlockState(CompressionAlgorithm compressionAlgo, RateLimitedOutputStream fsOut, + public WBlockState(CompressionAlgorithm compressionAlgo, FSDataOutputStream fsOut, BytesWritable fsOutputBuffer, Configuration conf, FileEncrypter encrypter) throws IOException { this.compressAlgo = compressionAlgo; this.fsOut = fsOut; - this.posStart = fsOut.position(); + this.posStart = fsOut.getPos(); fsOutputBuffer.setCapacity(getFSOutputBufferSize(conf)); @@ -174,7 +172,7 @@ OutputStream getOutputStream() { * @return The current byte offset in underlying file. */ long getCurrentPos() { - return fsOut.position() + fsBufferedOutput.size(); + return fsOut.getPos() + fsBufferedOutput.size(); } long getStartPos() { @@ -311,13 +309,13 @@ public void close() throws IOException { * blocks. * @see Compression#getSupportedAlgorithms */ - public Writer(FSDataOutputStream fout, RateLimiter writeLimiter, String compressionName, - Configuration conf, CryptoService cryptoService) throws IOException { + public Writer(FSDataOutputStream fout, String compressionName, Configuration conf, + CryptoService cryptoService) throws IOException { if (fout.getPos() != 0) { throw new IOException("Output file not at zero offset."); } - this.out = new RateLimitedOutputStream(fout, writeLimiter); + this.out = fout; this.conf = conf; dataIndex = new DataIndex(compressionName); metaIndex = new MetaIndex(); @@ -349,10 +347,10 @@ public void close() throws IOException { dataIndex.write(appender); } - long offsetIndexMeta = out.position(); + long offsetIndexMeta = out.getPos(); metaIndex.write(out); - long offsetCryptoParameter = out.position(); + long offsetCryptoParameter = out.getPos(); byte[] cryptoParams = this.encrypter.getDecryptionParameters(); out.writeInt(cryptoParams.length); out.write(cryptoParams); @@ -362,7 +360,7 @@ public void close() throws IOException { API_VERSION_3.write(out); Magic.write(out); out.flush(); - length = out.position(); + length = out.getPos(); out.close(); } } finally { diff --git a/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedInputStream.java b/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedInputStream.java deleted file mode 100644 index 45d00a473b9..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedInputStream.java +++ /dev/null @@ -1,73 +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.core.file.streams; - -import java.io.FilterInputStream; -import java.io.IOException; -import java.io.InputStream; - -import org.apache.accumulo.core.util.ratelimit.NullRateLimiter; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; -import org.apache.hadoop.fs.Seekable; - -/** - * A decorator for an {@code InputStream} which limits the rate at which reads are performed. - */ -public class RateLimitedInputStream extends FilterInputStream implements Seekable { - private final RateLimiter rateLimiter; - - public RateLimitedInputStream(StreamType stream, - RateLimiter rateLimiter) { - super(stream); - this.rateLimiter = rateLimiter == null ? NullRateLimiter.INSTANCE : rateLimiter; - } - - @Override - public int read() throws IOException { - int val = in.read(); - if (val >= 0) { - rateLimiter.acquire(1); - } - return val; - } - - @Override - public int read(byte[] buffer, int offset, int length) throws IOException { - int count = in.read(buffer, offset, length); - if (count > 0) { - rateLimiter.acquire(count); - } - return count; - } - - @Override - public void seek(long pos) throws IOException { - ((Seekable) in).seek(pos); - } - - @Override - public long getPos() throws IOException { - return ((Seekable) in).getPos(); - } - - @Override - public boolean seekToNewSource(long targetPos) throws IOException { - return ((Seekable) in).seekToNewSource(targetPos); - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStream.java b/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStream.java deleted file mode 100644 index 3ac7a761e7e..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStream.java +++ /dev/null @@ -1,60 +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.core.file.streams; - -import java.io.DataOutputStream; -import java.io.IOException; - -import org.apache.accumulo.core.util.ratelimit.NullRateLimiter; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; -import org.apache.hadoop.fs.FSDataOutputStream; - -/** - * A decorator for {@code OutputStream} which limits the rate at which data may be written. - * Underlying OutputStream is a FSDataOutputStream. - */ -public class RateLimitedOutputStream extends DataOutputStream { - private final RateLimiter writeLimiter; - - public RateLimitedOutputStream(FSDataOutputStream fsDataOutputStream, RateLimiter writeLimiter) { - super(fsDataOutputStream); - this.writeLimiter = writeLimiter == null ? NullRateLimiter.INSTANCE : writeLimiter; - } - - @Override - public synchronized void write(int i) throws IOException { - writeLimiter.acquire(1); - out.write(i); - } - - @Override - public synchronized void write(byte[] buffer, int offset, int length) throws IOException { - writeLimiter.acquire(length); - out.write(buffer, offset, length); - } - - @Override - public void close() throws IOException { - out.close(); - } - - public long position() { - return ((FSDataOutputStream) out).getPos(); - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java index d665ea977e9..793e5b5d9b1 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/Ample.java @@ -448,11 +448,6 @@ interface ConditionalTabletMutator extends TabletUpdates summarySelector, SummarizerFactory factory, CryptoService cryptoService) throws IOException { - CachableBuilder cb = new CachableBuilder().input(source.getInputStream(), cacheId) - .length(source.getLength()).conf(conf).cryptoService(cryptoService); + CachableBuilder cb = + new CachableBuilder().input((FSDataInputStream) source.getInputStream(), cacheId) + .length(source.getLength()).conf(conf).cryptoService(cryptoService); return load(new CachableBlockFile.Reader(cb), summarySelector, factory); } diff --git a/core/src/main/java/org/apache/accumulo/core/tabletserver/UnloaderParamsImpl.java b/core/src/main/java/org/apache/accumulo/core/tabletserver/UnloaderParamsImpl.java index 7874b0cdb27..b8f751ea36a 100644 --- a/core/src/main/java/org/apache/accumulo/core/tabletserver/UnloaderParamsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/tabletserver/UnloaderParamsImpl.java @@ -22,7 +22,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.TabletId; @@ -38,12 +37,12 @@ public class UnloaderParamsImpl implements UnloaderParams { private final SortedMap online; private final Set unloads; - public UnloaderParamsImpl(TableId tid, ServiceEnvironment env, Map online, + public UnloaderParamsImpl(TableId tid, ServiceEnvironment env, Map online, Set unload) { this.tid = tid; this.env = env; this.online = new TreeMap<>(); - online.forEach((k, v) -> this.online.put(new TabletIdImpl(k), v.get())); + online.forEach((k, v) -> this.online.put(new TabletIdImpl(k), v)); this.unloads = unload; } diff --git a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java index 471cd28f8a2..300b1753028 100644 --- a/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java +++ b/core/src/main/java/org/apache/accumulo/core/util/compaction/CompactionServicesConfig.java @@ -42,15 +42,9 @@ public class CompactionServicesConfig { private final Map planners = new HashMap<>(); private final Map rateLimits = new HashMap<>(); private final Map> options = new HashMap<>(); - long defaultRateLimit; public static final CompactionServiceId DEFAULT_SERVICE = CompactionServiceId.of("default"); - private long getDefaultThroughput() { - return ConfigurationTypeHelper - .getMemoryAsBytes(Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT.getDefaultValue()); - } - private static Map getConfiguration(AccumuloConfiguration aconf) { return aconf.getAllPropertiesWithPrefix(Property.TSERV_COMPACTION_SERVICE_PREFIX); } @@ -84,8 +78,6 @@ private CompactionServicesConfig(Map configs, Predicate } }); - defaultRateLimit = getDefaultThroughput(); - var diff = Sets.difference(options.keySet(), planners.keySet()); if (!diff.isEmpty()) { @@ -94,10 +86,6 @@ private CompactionServicesConfig(Map configs, Predicate } } - public long getRateLimit(String serviceName) { - return getRateLimits().getOrDefault(serviceName, defaultRateLimit); - } - @Override public boolean equals(Object o) { if (o instanceof CompactionServicesConfig) { diff --git a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/GuavaRateLimiter.java b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/GuavaRateLimiter.java deleted file mode 100644 index c2db7b0e9a6..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/GuavaRateLimiter.java +++ /dev/null @@ -1,66 +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.core.util.ratelimit; - -/** Rate limiter from the Guava library. */ -public class GuavaRateLimiter implements RateLimiter { - private final com.google.common.util.concurrent.RateLimiter rateLimiter; - private long currentRate; - - /** - * Constructor - * - * @param initialRate Count of permits which should be made available per second. A non-positive - * rate is taken to indicate there should be no limitation on rate. - */ - public GuavaRateLimiter(long initialRate) { - this.currentRate = initialRate; - this.rateLimiter = com.google.common.util.concurrent.RateLimiter - .create(initialRate > 0 ? initialRate : Long.MAX_VALUE); - } - - @Override - public long getRate() { - return currentRate; - } - - /** - * Change the rate at which permits are made available. - * - * @param newRate Count of permits which should be made available per second. A non-positive rate - * is taken to indicate that there should be no limitation on rate. - */ - public void setRate(long newRate) { - this.rateLimiter.setRate(newRate > 0 ? newRate : Long.MAX_VALUE); - this.currentRate = newRate; - } - - @Override - public void acquire(long numPermits) { - if (this.currentRate > 0) { - while (numPermits > Integer.MAX_VALUE) { - rateLimiter.acquire(Integer.MAX_VALUE); - numPermits -= Integer.MAX_VALUE; - } - if (numPermits > 0) { - rateLimiter.acquire((int) numPermits); - } - } - } -} diff --git a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/NullRateLimiter.java b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/NullRateLimiter.java deleted file mode 100644 index 8fae1479edd..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/NullRateLimiter.java +++ /dev/null @@ -1,37 +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.core.util.ratelimit; - -/** - * A rate limiter which doesn't actually limit rates at all. - */ -public class NullRateLimiter implements RateLimiter { - public static final NullRateLimiter INSTANCE = new NullRateLimiter(); - - private NullRateLimiter() {} - - @Override - public long getRate() { - return 0; - } - - @Override - public void acquire(long numPermits) {} - -} diff --git a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/RateLimiter.java b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/RateLimiter.java deleted file mode 100644 index b94ecb2638b..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/RateLimiter.java +++ /dev/null @@ -1,29 +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.core.util.ratelimit; - -public interface RateLimiter { - /** - * Get current QPS of the rate limiter, with a non-positive rate indicating no limit. - */ - long getRate(); - - /** Sleep until the specified number of queries are available. */ - void acquire(long numPermits); -} diff --git a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java b/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java deleted file mode 100644 index 71e26cde196..00000000000 --- a/core/src/main/java/org/apache/accumulo/core/util/ratelimit/SharedRateLimiterFactory.java +++ /dev/null @@ -1,193 +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.core.util.ratelimit; - -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.NANOSECONDS; - -import java.lang.ref.WeakReference; -import java.util.HashMap; -import java.util.Map; -import java.util.WeakHashMap; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Consumer; - -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.util.threads.ThreadPools; -import org.apache.accumulo.core.util.threads.Threads; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Provides the ability to retrieve a {@link RateLimiter} keyed to a specific string, which will - * dynamically update its rate according to a specified callback function. - */ -public class SharedRateLimiterFactory { - private static final long REPORT_RATE = 60000; - private static final long UPDATE_RATE = 1000; - private static SharedRateLimiterFactory instance = null; - private static ScheduledFuture updateTaskFuture; - private final Logger log = LoggerFactory.getLogger(SharedRateLimiterFactory.class); - private final WeakHashMap> activeLimiters = - new WeakHashMap<>(); - - private SharedRateLimiterFactory() {} - - /** Get the singleton instance of the SharedRateLimiterFactory. */ - public static synchronized SharedRateLimiterFactory getInstance(AccumuloConfiguration conf) { - if (instance == null) { - instance = new SharedRateLimiterFactory(); - - ScheduledThreadPoolExecutor svc = - ThreadPools.getServerThreadPools().createGeneralScheduledExecutorService(conf); - updateTaskFuture = svc.scheduleWithFixedDelay(Threads - .createNamedRunnable("SharedRateLimiterFactory update polling", instance::updateAll), - UPDATE_RATE, UPDATE_RATE, MILLISECONDS); - - ScheduledFuture future = svc.scheduleWithFixedDelay(Threads - .createNamedRunnable("SharedRateLimiterFactory report polling", instance::reportAll), - REPORT_RATE, REPORT_RATE, MILLISECONDS); - ThreadPools.watchNonCriticalScheduledTask(future); - - } - return instance; - } - - /** - * A callback which provides the current rate for a {@link RateLimiter}. - */ - public interface RateProvider { - /** - * Calculate the current rate for the {@link RateLimiter}. - * - * @return Count of permits which should be provided per second. A non-positive count is taken - * to indicate that no rate limiting should be performed. - */ - long getDesiredRate(); - } - - /** - * Lookup the RateLimiter associated with the specified name, or create a new one for that name. - * - * @param name key for the rate limiter - * @param rateProvider a function which can be called to get what the current rate for the rate - * limiter should be. - */ - public RateLimiter create(String name, RateProvider rateProvider) { - synchronized (activeLimiters) { - if (updateTaskFuture.isDone()) { - log.warn("SharedRateLimiterFactory update task has failed."); - } - var limiterRef = activeLimiters.get(name); - var limiter = limiterRef == null ? null : limiterRef.get(); - if (limiter == null) { - limiter = new SharedRateLimiter(name, rateProvider, rateProvider.getDesiredRate()); - activeLimiters.put(name, new WeakReference<>(limiter)); - } - return limiter; - } - } - - private void copyAndThen(String actionName, Consumer action) { - Map limitersCopy = new HashMap<>(); - // synchronize only for copy - synchronized (activeLimiters) { - activeLimiters.forEach((name, limiterRef) -> { - var limiter = limiterRef.get(); - if (limiter != null) { - limitersCopy.put(name, limiter); - } - }); - } - limitersCopy.forEach((name, limiter) -> { - try { - action.accept(limiter); - } catch (RuntimeException e) { - log.error("Failed to {} limiter {}", actionName, name, e); - } - }); - } - - /** - * Walk through all of the currently active RateLimiters, having each update its current rate. - * This is called periodically so that we can dynamically update as configuration changes. - */ - private void updateAll() { - copyAndThen("update", SharedRateLimiter::update); - } - - /** - * Walk through all of the currently active RateLimiters, having each report its activity to the - * debug log. - */ - private void reportAll() { - copyAndThen("report", SharedRateLimiter::report); - } - - protected class SharedRateLimiter extends GuavaRateLimiter { - private AtomicLong permitsAcquired = new AtomicLong(); - private AtomicLong lastUpdate = new AtomicLong(); - - private final RateProvider rateProvider; - private final String name; - - SharedRateLimiter(String name, RateProvider rateProvider, long initialRate) { - super(initialRate); - this.name = name; - this.rateProvider = rateProvider; - this.lastUpdate.set(System.nanoTime()); - } - - @Override - public void acquire(long numPermits) { - super.acquire(numPermits); - permitsAcquired.addAndGet(numPermits); - } - - /** Poll the callback, updating the current rate if necessary. */ - public void update() { - // Reset rate if needed - long rate = rateProvider.getDesiredRate(); - if (rate != getRate()) { - setRate(rate); - } - } - - /** Report the current throughput and usage of this rate limiter to the debug log. */ - public void report() { - if (log.isDebugEnabled()) { - long duration = NANOSECONDS.toMillis(System.nanoTime() - lastUpdate.get()); - if (duration == 0) { - return; - } - lastUpdate.set(System.nanoTime()); - - long sum = permitsAcquired.get(); - permitsAcquired.set(0); - - if (sum > 0) { - log.debug(String.format("RateLimiter '%s': %,d of %,d permits/second", name, - sum * 1000L / duration, getRate())); - } - } - } - } -} diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java index a220407b92f..d7b31bc73d3 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/AbstractRFileTest.java @@ -99,7 +99,7 @@ public void openWriter(boolean startDLG, int blockSize) throws IOException { CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE, accumuloConfiguration.getAllCryptoProperties()); - BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, cs); + BCFile.Writer _cbw = new BCFile.Writer(dos, "gz", conf, cs); SamplerConfigurationImpl samplerConfig = SamplerConfigurationImpl.newSamplerConfig(accumuloConfiguration); diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java index cd13745a41d..29926b0191e 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/CreateCompatTestFile.java @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { Configuration conf = new Configuration(); FileSystem fs = FileSystem.get(conf); AccumuloConfiguration aconf = DefaultConfiguration.getInstance(); - BCFile.Writer _cbw = new BCFile.Writer(fs.create(new Path(args[0])), null, "gz", conf, + BCFile.Writer _cbw = new BCFile.Writer(fs.create(new Path(args[0])), "gz", conf, CryptoFactoryLoader.getServiceForServer(aconf)); RFile.Writer writer = new RFile.Writer(_cbw, 1000); diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java index eb9a81bf0be..3698216bead 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/FencedRFileTest.java @@ -20,21 +20,26 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.crypto.CryptoTest; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.file.FileSKVIterator; +import org.apache.accumulo.core.file.rfile.RFile.FencedIndex; import org.apache.accumulo.core.file.rfile.RFile.FencedReader; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iteratorsImpl.system.IterationInterruptedException; import org.apache.accumulo.core.iteratorsImpl.system.MultiIterator; +import org.apache.accumulo.core.sample.impl.SamplerConfigurationImpl; import org.apache.hadoop.io.Text; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -173,10 +178,7 @@ public void testFencing11() throws IOException { @Test public void testFencing12() throws IOException { - final TestRFile trf = new TestRFile(conf); - trf.openWriter(); - writeTestFile(trf); - trf.closeWriter(); + final TestRFile trf = initTestFile(); // Fence off the file to contain only 1 row (r_00001) Range range = new Range(new Range("r_000001")); @@ -205,6 +207,111 @@ public void testFencing12() throws IOException { assertFalse(iter.hasTop()); } + @Test + public void testFirstAndLastRow() throws IOException { + final TestRFile trf = initTestFile(); + + Text firstRowInFile = new Text(formatString("r_", 0)); + Text lastRowInFile = new Text(formatString("r_", 3)); + + // Infinite range fence + // Should just be first/last rows of file + assertReader(trf, new Range(), (reader) -> { + assertEquals(firstRowInFile, reader.getFirstRow()); + assertEquals(lastRowInFile, reader.getLastRow()); + }); + + // Range inside of file so should return the rows of the fence + assertReader(trf, new Range("r_000001", "r_000002"), (reader) -> { + assertEquals(new Text("r_000001"), reader.getFirstRow()); + assertEquals(new Text("r_000002"), reader.getLastRow()); + }); + + // Test infinite start row + assertReader(trf, new Range(null, "r_000001"), (reader) -> { + assertEquals(firstRowInFile, reader.getFirstRow()); + assertEquals(new Text("r_000001"), reader.getLastRow()); + }); + + // Test infinite end row + assertReader(trf, new Range("r_000002", null), (reader) -> { + assertEquals(new Text("r_000002"), reader.getFirstRow()); + assertEquals(lastRowInFile, reader.getLastRow()); + }); + + // Test start row matches start of file + assertReader(trf, new Range("r_000000", "r_000002"), (reader) -> { + // start row of range matches first row in file so that should be returned instead + assertEquals(firstRowInFile, reader.getFirstRow()); + assertEquals(new Text("r_000002"), reader.getLastRow()); + }); + + // Test end row matches end of file + assertReader(trf, new Range("r_000001", "r_000003"), (reader) -> { + assertEquals(new Text("r_000001"), reader.getFirstRow()); + // end row of range matches last row in file so that should be returned instead + assertEquals(lastRowInFile, reader.getLastRow()); + }); + + // Test case where rows in range are less than and greater than rows in file + assertReader(trf, new Range("a", "z"), (reader) -> { + assertEquals(firstRowInFile, reader.getFirstRow()); + assertEquals(lastRowInFile, reader.getLastRow()); + }); + + // Test inclusive end key, usually a row range is required to be an exclusive key + // for a tablet file but the fenced reader still supports any range type + assertReader(trf, new Range(new Key("r_000002"), true, new Key("r_000002"), true), (reader) -> { + assertEquals(new Text("r_000002"), reader.getFirstRow()); + assertEquals(new Text("r_000002"), reader.getLastRow()); + }); + + } + + @Test + public void testUnsupportedMethods() throws IOException { + final TestRFile trf = initTestFile(); + trf.openReader(new Range()); + FencedReader reader = (FencedReader) trf.iter; + FencedIndex index = (FencedIndex) reader.getIndex(); + + assertThrows(UnsupportedOperationException.class, () -> reader.init(null, null, null)); + assertThrows(UnsupportedOperationException.class, + () -> index.getSample(new SamplerConfigurationImpl())); + assertThrows(UnsupportedOperationException.class, + () -> index.seek(new Range(), List.of(), false)); + assertThrows(UnsupportedOperationException.class, () -> index.deepCopy(null)); + } + + @Test + public void testSetInterrupted() throws IOException { + final TestRFile trf = initTestFile(); + trf.openReader(new Range()); + FencedReader reader = (FencedReader) trf.iter; + + reader.setInterruptFlag(new AtomicBoolean(true)); + assertThrows(IterationInterruptedException.class, + () -> reader.seek(new Range("r_000001"), List.of(), false)); + + } + + @Test + public void testReset() throws IOException { + final TestRFile trf = initTestFile(); + trf.openReader(new Range()); + FencedReader reader = (FencedReader) trf.iter; + + assertFalse(reader.hasTop()); + reader.seek(new Range("r_000001"), List.of(), false); + assertTrue(reader.hasTop()); + assertEquals( + newKey(formatString("r_", 1), formatString("cf_", 0), formatString("cq_", 0), "A", 4), + reader.getTopKey()); + + reader.reset(); + assertFalse(reader.hasTop()); + } + private int testFencing(List fencedRange, List expectedRange) throws IOException { // test an rfile with multiple rows having multiple columns @@ -370,4 +477,33 @@ private void writeTestFile(final TestRFile trf, final List expectedKeys, } } } + + private TestRFile initTestFile() throws IOException { + final TestRFile trf = new TestRFile(conf); + trf.openWriter(); + writeTestFile(trf); + trf.closeWriter(); + return trf; + } + + private static void assertReader(final TestRFile trf, Range range, + ThrowableConsumer run) throws IOException { + FencedReader reader = null; + try { + trf.openReader(range); + reader = (FencedReader) trf.iter; + run.accept(reader); + } finally { + if (reader != null) { + reader.close(); + } + } + + } + + // Similar to the java.util.function.Consumer interface but throws an exception + interface ThrowableConsumer { + void accept(T t) throws U; + } + } diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java index 2065f8b6761..b0d2bb0c3e2 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiLevelIndexTest.java @@ -66,7 +66,7 @@ private void runTest(int maxBlockSize, int num) throws IOException { ByteArrayOutputStream baos = new ByteArrayOutputStream(); FSDataOutputStream dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a")); CryptoService cs = CryptoFactoryLoader.getServiceForServer(aconf); - BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", hadoopConf, cs); + BCFile.Writer _cbw = new BCFile.Writer(dos, "gz", hadoopConf, cs); BufferedWriter mliw = new BufferedWriter(new Writer(_cbw, maxBlockSize)); diff --git a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java index 8c66bb6ead6..5e7f237eb36 100644 --- a/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java +++ b/core/src/test/java/org/apache/accumulo/core/file/rfile/MultiThreadedRFileTest.java @@ -153,7 +153,7 @@ public void openWriter(boolean startDLG) throws IOException { FileSystem fs = FileSystem.newInstance(conf); Path path = new Path("file://" + rfile); dos = fs.create(path, true); - BCFile.Writer _cbw = new BCFile.Writer(dos, null, "gz", conf, + BCFile.Writer _cbw = new BCFile.Writer(dos, "gz", conf, CryptoFactoryLoader.getServiceForServer(accumuloConfiguration)); SamplerConfigurationImpl samplerConfig = SamplerConfigurationImpl.newSamplerConfig(accumuloConfiguration); diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java deleted file mode 100644 index 8eb3a5ef673..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedInputStreamTest.java +++ /dev/null @@ -1,83 +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.core.file.streams; - -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.io.InputStream; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.accumulo.core.util.ratelimit.RateLimiter; -import org.apache.hadoop.fs.Seekable; -import org.easymock.EasyMock; -import org.junit.jupiter.api.Test; - -public class RateLimitedInputStreamTest { - - @Test - public void permitsAreProperlyAcquired() throws Exception { - // Create variables for tracking behaviors of mock object - AtomicLong rateLimiterPermitsAcquired = new AtomicLong(); - // Construct mock object - RateLimiter rateLimiter = EasyMock.niceMock(RateLimiter.class); - // Stub Mock Method - rateLimiter.acquire(EasyMock.anyLong()); - EasyMock.expectLastCall() - .andAnswer(() -> rateLimiterPermitsAcquired.addAndGet(EasyMock.getCurrentArgument(0))) - .anyTimes(); - EasyMock.replay(rateLimiter); - - long bytesRetrieved = 0; - try (InputStream is = new RateLimitedInputStream(new RandomInputStream(), rateLimiter)) { - for (int i = 0; i < 100; ++i) { - int count = Math.abs(RANDOM.get().nextInt()) % 65536; - int countRead = is.read(new byte[count]); - assertEquals(count, countRead); - bytesRetrieved += count; - } - } - assertEquals(bytesRetrieved, rateLimiterPermitsAcquired.get()); - } - - private static class RandomInputStream extends InputStream implements Seekable { - - @Override - public int read() { - return RANDOM.get().nextInt() & 0xff; - } - - @Override - public void seek(long pos) { - throw new UnsupportedOperationException("Not supported yet."); - } - - @Override - public long getPos() { - throw new UnsupportedOperationException("Not supported yet."); - } - - @Override - public boolean seekToNewSource(long targetPos) { - throw new UnsupportedOperationException("Not supported yet."); - } - - } - -} diff --git a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java b/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java deleted file mode 100644 index 8df1a3104e3..00000000000 --- a/core/src/test/java/org/apache/accumulo/core/file/streams/RateLimitedOutputStreamTest.java +++ /dev/null @@ -1,68 +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.core.file.streams; - -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.io.OutputStream; -import java.util.concurrent.atomic.AtomicLong; - -import org.apache.accumulo.core.util.ratelimit.RateLimiter; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.easymock.EasyMock; -import org.junit.jupiter.api.Test; - -import com.google.common.io.CountingOutputStream; - -public class RateLimitedOutputStreamTest { - - @Test - public void permitsAreProperlyAcquired() throws Exception { - // Create variables for tracking behaviors of mock object - AtomicLong rateLimiterPermitsAcquired = new AtomicLong(); - // Construct mock object - RateLimiter rateLimiter = EasyMock.niceMock(RateLimiter.class); - // Stub Mock Method - rateLimiter.acquire(EasyMock.anyLong()); - EasyMock.expectLastCall() - .andAnswer(() -> rateLimiterPermitsAcquired.addAndGet(EasyMock.getCurrentArgument(0))) - .anyTimes(); - EasyMock.replay(rateLimiter); - - long bytesWritten = 0; - try (RateLimitedOutputStream os = - new RateLimitedOutputStream(new NullOutputStream(), rateLimiter)) { - for (int i = 0; i < 100; ++i) { - byte[] bytes = new byte[Math.abs(RANDOM.get().nextInt() % 65536)]; - os.write(bytes); - bytesWritten += bytes.length; - } - assertEquals(bytesWritten, os.position()); - } - assertEquals(bytesWritten, rateLimiterPermitsAcquired.get()); - } - - public static class NullOutputStream extends FSDataOutputStream { - public NullOutputStream() { - super(new CountingOutputStream(OutputStream.nullOutputStream()), null); - } - } - -} diff --git a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java index 1c5b35e81fc..0c5bad07c47 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java +++ b/server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java @@ -64,7 +64,6 @@ import org.apache.accumulo.core.trace.TraceUtil; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.core.util.LocalityGroupUtil.LocalityGroupConfigurationError; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; @@ -94,10 +93,6 @@ public interface CompactionEnv { IteratorScope getIteratorScope(); - RateLimiter getReadLimiter(); - - RateLimiter getWriteLimiter(); - SystemIteratorEnvironment createIteratorEnv(ServerContext context, AccumuloConfiguration acuTableConf, TableId tableId); @@ -233,7 +228,7 @@ public CompactionStats call() throws IOException, CompactionCanceledException { WriterBuilder outBuilder = fileFactory.newWriterBuilder().forFile(outputFile, ns, ns.getConf(), cryptoService) - .withTableConfiguration(acuTableConf).withRateLimiter(env.getWriteLimiter()); + .withTableConfiguration(acuTableConf); if (dropCacheBehindOutput) { outBuilder.dropCachesBehind(); } @@ -338,8 +333,7 @@ public CompactionStats call() throws IOException, CompactionCanceledException { FileSKVIterator reader; reader = fileFactory.newReaderBuilder().forFile(dataFile, fs, fs.getConf(), cryptoService) - .withTableConfiguration(acuTableConf).withRateLimiter(env.getReadLimiter()) - .dropCachesBehind().build(); + .withTableConfiguration(acuTableConf).dropCachesBehind().build(); readers.add(reader); diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java index e84748f324c..f1fa558dd17 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/AbstractTabletStateStore.java @@ -51,7 +51,6 @@ public void setLocations(Collection assignments) throws DistributedS for (Assignment assignment : assignments) { var conditionalMutator = tabletsMutator.mutateTablet(assignment.tablet) .requireLocation(TabletMetadata.Location.future(assignment.server)) - .requirePrevEndRow(assignment.tablet.prevEndRow()) .putLocation(TabletMetadata.Location.current(assignment.server)) .deleteLocation(TabletMetadata.Location.future(assignment.server)).deleteSuspension(); @@ -81,8 +80,8 @@ public void setFutureLocations(Collection assignments) try (var tabletsMutator = ample.conditionallyMutateTablets()) { for (Assignment assignment : assignments) { tabletsMutator.mutateTablet(assignment.tablet).requireAbsentOperation() - .requireAbsentLocation().requirePrevEndRow(assignment.tablet.prevEndRow()) - .deleteSuspension().putLocation(TabletMetadata.Location.future(assignment.server)) + .requireAbsentLocation().deleteSuspension() + .putLocation(TabletMetadata.Location.future(assignment.server)) .submit(tabletMetadata -> { Preconditions.checkArgument(tabletMetadata.getExtent().equals(assignment.tablet)); return tabletMetadata.getLocation() != null && tabletMetadata.getLocation() @@ -127,8 +126,8 @@ private void unassign(Collection tablets, continue; } - var tabletMutator = tabletsMutator.mutateTablet(tm.getExtent()) - .requireLocation(tm.getLocation()).requirePrevEndRow(tm.getExtent().prevEndRow()); + var tabletMutator = + tabletsMutator.mutateTablet(tm.getExtent()).requireLocation(tm.getLocation()); if (tm.hasCurrent()) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java index fc15a0b39ca..9bf6616bdfc 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MergeState.java @@ -23,10 +23,6 @@ public enum MergeState { * Not merging */ NONE, - /** - * created, stored in zookeeper, other merges are prevented on the table - */ - STARTED, /** * when the number of chopped tablets in the range matches the number of online tablets in the * range, take the tablets offline diff --git a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java index d69c114f5bf..2a2ee11c095 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java +++ b/server/base/src/main/java/org/apache/accumulo/server/manager/state/MetaDataStateStore.java @@ -71,8 +71,7 @@ public void unsuspend(Collection tablets) throws DistributedStor } // ELASTICITY_TODO pending #3314, add conditional mutation check that tls.suspend exists - tabletsMutator.mutateTablet(tm.getExtent()).requireAbsentOperation() - .requirePrevEndRow(tm.getExtent().prevEndRow()).deleteSuspension() + tabletsMutator.mutateTablet(tm.getExtent()).requireAbsentOperation().deleteSuspension() .submit(tabletMetadata -> tabletMetadata.getSuspend() == null); } diff --git a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java index 4393a091107..a77a357d1f0 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java +++ b/server/base/src/main/java/org/apache/accumulo/server/metadata/ConditionalTabletMutatorImpl.java @@ -22,6 +22,7 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN; import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.COMPACT_COLUMN; +import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.FLUSH_COLUMN; import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.OPID_COLUMN; import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.SELECTED_COLUMN; import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN; @@ -56,7 +57,6 @@ import org.apache.accumulo.server.metadata.iterators.PresentIterator; import org.apache.accumulo.server.metadata.iterators.SetEqualityIterator; import org.apache.accumulo.server.metadata.iterators.TabletExistsIterator; -import org.apache.hadoop.io.Text; import com.google.common.base.Preconditions; @@ -74,6 +74,7 @@ public class ConditionalTabletMutatorImpl extends TabletMutatorBase mutationConsumer, @@ -107,16 +108,6 @@ public Ample.ConditionalTabletMutator requireLocation(Location location) { return this; } - @Override - public Ample.ConditionalTabletMutator requirePrevEndRow(Text per) { - Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); - Condition c = - new Condition(PREV_ROW_COLUMN.getColumnFamily(), PREV_ROW_COLUMN.getColumnQualifier()) - .setValue(encodePrevEndRow(per).get()); - mutation.addCondition(c); - return this; - } - @Override public Ample.ConditionalTabletMutator requireHostingGoal(TabletHostingGoal goal) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); @@ -143,6 +134,7 @@ public Ample.ConditionalTabletMutator requireAbsentTablet() { Condition c = new Condition("", "").setIterators(is); mutation.addCondition(c); sawOperationRequirement = true; + checkPrevEndRow = false; return this; } @@ -168,8 +160,7 @@ public Ample.ConditionalTabletMutator requireOperation(TabletOperationId opid) { private void requireSameSingle(TabletMetadata tabletMetadata, ColumnType type) { switch (type) { case PREV_ROW: - requirePrevEndRow(tabletMetadata.getPrevEndRow()); - break; + throw new IllegalStateException("PREV_ROW already set from Extent"); case COMPACT_ID: { Condition c = new Condition(COMPACT_COLUMN.getColumnFamily(), COMPACT_COLUMN.getColumnQualifier()); @@ -231,6 +222,15 @@ private void requireSameSingle(TabletMetadata tabletMetadata, ColumnType type) { mutation.addCondition(c); } break; + case FLUSH_ID: { + Condition c = + new Condition(FLUSH_COLUMN.getColumnFamily(), FLUSH_COLUMN.getColumnQualifier()); + if (tabletMetadata.getFlushId().isPresent()) { + c = c.setValue(Long.toString(tabletMetadata.getFlushId().getAsLong())); + } + mutation.addCondition(c); + } + break; default: throw new UnsupportedOperationException("Column type " + type + " is not supported."); } @@ -251,6 +251,12 @@ public Ample.ConditionalTabletMutator requireSame(TabletMetadata tabletMetadata, public void submit(Ample.RejectionHandler rejectionCheck) { Preconditions.checkState(updatesEnabled, "Cannot make updates after calling mutate."); Preconditions.checkState(sawOperationRequirement, "No operation requirements were seen"); + if (checkPrevEndRow) { + Condition c = + new Condition(PREV_ROW_COLUMN.getColumnFamily(), PREV_ROW_COLUMN.getColumnQualifier()) + .setValue(encodePrevEndRow(extent.prevEndRow()).get()); + mutation.addCondition(c); + } getMutation(); mutationConsumer.accept(mutation); rejectionHandlerConsumer.accept(extent, rejectionCheck); diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java index f109991cc19..69e7da8e6eb 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/ManagerMetadataUtil.java @@ -24,8 +24,6 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Optional; -import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; @@ -179,37 +177,6 @@ private static KeyExtent fixSplit(ServerContext context, TableId tableId, Text m } } - /** - * Update tablet file data from flush. Returns a StoredTabletFile if there are data entries. - */ - public static Optional updateTabletDataFile(ServerContext context, - KeyExtent extent, ReferencedTabletFile newDatafile, DataFileValue dfv, MetadataTime time, - TServerInstance tServerInstance, ServiceLock zooLock, Set unusedWalLogs, - Location lastLocation, long flushId) { - - // ELASTICITY_TODO use conditional mutation and require tablet location - TabletMutator tablet = context.getAmple().mutateTablet(extent); - // if there are no entries, the path doesn't get stored in metadata table, only the flush ID - Optional newFile = Optional.empty(); - - // if entries are present, write to path to metadata table - if (dfv.getNumEntries() > 0) { - tablet.putFile(newDatafile, dfv); - tablet.putTime(time); - newFile = Optional.of(newDatafile.insert()); - - updateLastForCompactionMode(context, tablet, lastLocation, tServerInstance); - } - tablet.putFlushId(flushId); - - unusedWalLogs.forEach(tablet::deleteWal); - - tablet.putZooLock(context.getZooKeeperRoot(), zooLock); - - tablet.mutate(); - return newFile; - } - /** * Update the last location if the location mode is "assignment". This will delete the previous * last location if needed and set the new last location @@ -240,8 +207,9 @@ public static void updateLastForAssignmentMode(ClientContext context, * @param lastLocation The last location * @param tServerInstance The server address */ - public static void updateLastForCompactionMode(ClientContext context, TabletMutator tabletMutator, - Location lastLocation, TServerInstance tServerInstance) { + public static void updateLastForCompactionMode(ClientContext context, + Ample.ConditionalTabletMutator tabletMutator, Location lastLocation, + TServerInstance tServerInstance) { // if the location mode is 'compaction', then preserve the current compaction location in the // last location value if ("compaction".equals(context.getConfiguration().get(Property.TSERV_LAST_LOCATION_MODE))) { diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java index 1a2db0d1065..c6b2e8689dd 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java +++ b/server/base/src/main/java/org/apache/accumulo/server/util/MetadataTableUtil.java @@ -141,14 +141,6 @@ private static void logUpdateFailure(Mutation m, KeyExtent extent, Exception e) log.error("Failed to write metadata updates for extent {} {}", extent, m.prettyPrint(), e); } - public static void updateTabletFlushID(KeyExtent extent, long flushID, ServerContext context, - ServiceLock zooLock) { - TabletMutator tablet = context.getAmple().mutateTablet(extent); - tablet.putFlushId(flushID); - tablet.putZooLock(context.getZooKeeperRoot(), zooLock); - tablet.mutate(); - } - public static Map updateTabletDataFile(long tid, KeyExtent extent, Map estSizes, MetadataTime time, ServerContext context, ServiceLock zooLock) { diff --git a/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java b/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java index 01ae58bf24c..2d7a3a79275 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/manager/state/MergeInfoTest.java @@ -76,7 +76,7 @@ public void testSerialization() throws Exception { Text prevEndRow = new Text("begin"); keyExtent = new KeyExtent(TableId.of(table), endRow, prevEndRow); mi = new MergeInfo(keyExtent, MergeInfo.Operation.DELETE); - mi.setState(MergeState.STARTED); + mi.setState(MergeState.WAITING_FOR_OFFLINE); ByteArrayOutputStream baos = new ByteArrayOutputStream(); DataOutputStream dos = new DataOutputStream(baos); mi.write(dos); @@ -84,7 +84,7 @@ public void testSerialization() throws Exception { DataInputStream dis = new DataInputStream(bais); mi = new MergeInfo(); mi.readFields(dis); - assertSame(MergeState.STARTED, mi.getState()); + assertSame(MergeState.WAITING_FOR_OFFLINE, mi.getState()); assertEquals(keyExtent, mi.getExtent()); assertSame(MergeInfo.Operation.DELETE, mi.getOperation()); } 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 59235b6cecc..ea1c5245e14 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 @@ -27,8 +27,6 @@ import org.apache.accumulo.core.spi.compaction.CompactionKind; import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason; import org.apache.accumulo.core.tabletserver.thrift.TExternalCompactionJob; -import org.apache.accumulo.core.util.ratelimit.NullRateLimiter; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.FileCompactor.CompactionEnv; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; @@ -74,16 +72,6 @@ public IteratorScope getIteratorScope() { return IteratorScope.majc; } - @Override - public RateLimiter getReadLimiter() { - return NullRateLimiter.INSTANCE; - } - - @Override - public RateLimiter getWriteLimiter() { - return NullRateLimiter.INSTANCE; - } - @Override public SystemIteratorEnvironment createIteratorEnv(ServerContext context, AccumuloConfiguration acuTableConf, TableId tableId) { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java index 21183349f5d..f10dbd25c1f 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/Manager.java @@ -539,7 +539,7 @@ public void setMergeState(MergeInfo info, MergeState state) throw new AssertionError("Unlikely", ex); } context.getZooReaderWriter().putPersistentData(path, out.getData(), - state.equals(MergeState.STARTED) ? ZooUtil.NodeExistsPolicy.FAIL + state.equals(MergeState.WAITING_FOR_OFFLINE) ? ZooUtil.NodeExistsPolicy.FAIL : ZooUtil.NodeExistsPolicy.OVERWRITE); } mergeLock.notifyAll(); @@ -711,8 +711,6 @@ TabletGoalState getGoalState(TabletMetadata tm, MergeInfo mergeInfo) { case NONE: case COMPLETE: break; - case STARTED: - return TabletGoalState.HOSTED; case WAITING_FOR_OFFLINE: case MERGING: return TabletGoalState.UNASSIGNED; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java index 81215dbdd65..97b9a07682f 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/ManagerClientServiceHandler.java @@ -630,8 +630,7 @@ public void requestTabletHosting(TInfo tinfo, TCredentials credentials, String t if (recentHostingRequest.getIfPresent(ke) == null) { mutator.mutateTablet(ke).requireAbsentOperation() .requireHostingGoal(TabletHostingGoal.ONDEMAND).requireAbsentLocation() - .requirePrevEndRow(ke.prevEndRow()).setHostingRequested() - .submit(TabletMetadata::getHostingRequested); + .setHostingRequested().submit(TabletMetadata::getHostingRequested); } else { log.trace("Ignoring hosting request because it was recently requested {}", ke); } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java index fbec61c1194..4528abc33e4 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/TabletGroupWatcher.java @@ -26,44 +26,29 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.SortedMap; import java.util.SortedSet; import java.util.TreeMap; -import java.util.TreeSet; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.function.Predicate; -import java.util.stream.Stream; -import org.apache.accumulo.core.client.AccumuloClient; -import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.BatchWriter; -import org.apache.accumulo.core.client.MutationsRejectedException; -import org.apache.accumulo.core.client.RowIterator; import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.client.admin.TabletHostingGoal; -import org.apache.accumulo.core.clientImpl.TabletHostingGoalUtil; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; -import org.apache.accumulo.core.data.PartialKey; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.gc.ReferenceFile; import org.apache.accumulo.core.logging.TabletLogger; import org.apache.accumulo.core.manager.balancer.TabletServerIdImpl; import org.apache.accumulo.core.manager.state.TabletManagement; @@ -74,41 +59,25 @@ import org.apache.accumulo.core.manager.thrift.TabletServerStatus; import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; -import org.apache.accumulo.core.metadata.StoredTabletFile; import org.apache.accumulo.core.metadata.TServerInstance; import org.apache.accumulo.core.metadata.TabletState; -import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; -import org.apache.accumulo.core.metadata.schema.DataFileValue; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ExternalCompactionColumnFamily; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.FutureLocationColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.HostingColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataTime; import org.apache.accumulo.core.metadata.schema.TabletMetadata; -import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location; -import org.apache.accumulo.core.metadata.schema.TabletsMetadata; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.spi.balancer.data.TabletServerId; -import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.TextUtil; import org.apache.accumulo.core.util.threads.Threads; import org.apache.accumulo.core.util.threads.Threads.AccumuloDaemonThread; import org.apache.accumulo.manager.Manager.TabletGoalState; import org.apache.accumulo.manager.split.SplitTask; -import org.apache.accumulo.manager.state.MergeStats; import org.apache.accumulo.manager.state.TableCounts; import org.apache.accumulo.manager.state.TableStats; -import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.ServiceEnvironmentImpl; import org.apache.accumulo.server.compaction.CompactionJobGenerator; import org.apache.accumulo.server.conf.TableConfiguration; -import org.apache.accumulo.server.gc.AllVolumesDirectory; import org.apache.accumulo.server.log.WalStateManager; import org.apache.accumulo.server.log.WalStateManager.WalMarkerException; import org.apache.accumulo.server.manager.LiveTServerSet.TServerConnection; @@ -116,23 +85,17 @@ import org.apache.accumulo.server.manager.state.ClosableIterator; import org.apache.accumulo.server.manager.state.DistributedStoreException; import org.apache.accumulo.server.manager.state.MergeInfo; -import org.apache.accumulo.server.manager.state.MergeState; import org.apache.accumulo.server.manager.state.TabletManagementIterator; import org.apache.accumulo.server.manager.state.TabletStateStore; import org.apache.accumulo.server.manager.state.UnassignedTablet; -import org.apache.accumulo.server.tablets.TabletTime; -import org.apache.accumulo.server.util.MetadataTableUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableSortedSet; import com.google.common.collect.Iterators; -import com.google.common.collect.Sets; abstract class TabletGroupWatcher extends AccumuloDaemonThread { @@ -327,8 +290,6 @@ synchronized void waitForFullScan(long millis) { private static class TableMgmtStats { int[] counts = new int[TabletState.values().length]; private int totalUnloaded; - - Map mergeStatsCache = new HashMap<>(); } private TableMgmtStats manageTablets(Iterator iter, @@ -348,10 +309,10 @@ private TableMgmtStats manageTablets(Iterator iter, TableMgmtStats tableMgmtStats = new TableMgmtStats(); int unloaded = 0; - Map currentMerges = new HashMap<>(); + Map currentMerges = new HashMap<>(); for (MergeInfo merge : manager.merges()) { if (merge.getExtent() != null) { - currentMerges.put(merge.getExtent().tableId(), new MergeStats(merge)); + currentMerges.put(merge.getExtent().tableId(), merge); } } @@ -400,11 +361,8 @@ private TableMgmtStats manageTablets(Iterator iter, final TableConfiguration tableConf = manager.getContext().getTableConfiguration(tableId); - final MergeStats mergeStats = tableMgmtStats.mergeStatsCache.computeIfAbsent(tableId, k -> { - var mStats = currentMerges.get(k); - return mStats != null ? mStats : new MergeStats(new MergeInfo()); - }); - TabletGoalState goal = manager.getGoalState(tm, mergeStats.getMergeInfo()); + TabletGoalState goal = manager.getGoalState(tm, + currentMerges.computeIfAbsent(tm.getTableId(), k -> new MergeInfo())); TabletState state = TabletState.compute(tm, currentTServers.keySet(), manager.tabletBalancer, resourceGroups); @@ -423,7 +381,6 @@ private TableMgmtStats manageTablets(Iterator iter, if (isFullScan) { stats.update(tableId, state); } - mergeStats.update(tm.getExtent(), state); // Always follow through with assignments if (state == TabletState.ASSIGNED) { @@ -644,8 +601,6 @@ public void run() { tabletMgmtStats.totalUnloaded); } - updateMergeState(tabletMgmtStats.mergeStatsCache); - synchronized (this) { lastScanServers = ImmutableSortedSet.copyOf(currentTServers.keySet()); } @@ -820,706 +775,6 @@ private int assignedOrHosted(Map last) { return result; } - private void updateMergeState(Map mergeStatsCache) { - for (MergeStats stats : mergeStatsCache.values()) { - try { - MergeState update = stats.nextMergeState(manager.getContext(), manager); - // when next state is MERGING, its important to persist this before - // starting the merge... the verification check that is done before - // moving into the merging state could fail if merge starts but does - // not finish - if (update == MergeState.COMPLETE) { - update = MergeState.NONE; - } - if (update != stats.getMergeInfo().getState()) { - manager.setMergeState(stats.getMergeInfo(), update); - } - - if (update == MergeState.MERGING) { - try { - if (stats.getMergeInfo().isDelete()) { - deleteTablets(stats.getMergeInfo()); - } else { - mergeMetadataRecords(stats.getMergeInfo()); - } - update = MergeState.COMPLETE; - manager.setMergeState(stats.getMergeInfo(), update); - } catch (Exception ex) { - Manager.log.error("Unable merge metadata table records", ex); - } - } - } catch (Exception ex) { - Manager.log.error( - "Unable to update merge state for merge " + stats.getMergeInfo().getExtent(), ex); - } - } - } - - // This method finds returns the deletion starting row (exclusive) for tablets that - // need to be actually deleted. If the startTablet is null then - // the deletion start row will just be null as all tablets are being deleted - // up to the end. Otherwise, this returns the endRow of the first tablet - // as the first tablet should be kept and will have been previously - // fenced if necessary - private Text getDeletionStartRow(final KeyExtent startTablet) { - if (startTablet == null) { - Manager.log.debug("First tablet for delete range is null"); - return null; - } - - final Text deletionStartRow = startTablet.endRow(); - Manager.log.debug("Start row is {} for deletion", deletionStartRow); - - return deletionStartRow; - } - - // This method finds returns the deletion ending row (inclusive) for tablets that - // need to be actually deleted. If the endTablet is null then - // the deletion end row will just be null as all tablets are being deleted - // after the start row. Otherwise, this returns the prevEndRow of the last tablet - // as the last tablet should be kept and will have been previously - // fenced if necessary - private Text getDeletionEndRow(final KeyExtent endTablet) { - if (endTablet == null) { - Manager.log.debug("Last tablet for delete range is null"); - return null; - } - - Text deletionEndRow = endTablet.prevEndRow(); - Manager.log.debug("Deletion end row is {}", deletionEndRow); - - return deletionEndRow; - } - - private static boolean isFirstTabletInTable(KeyExtent tablet) { - return tablet != null && tablet.prevEndRow() == null; - } - - private static boolean isLastTabletInTable(KeyExtent tablet) { - return tablet != null && tablet.endRow() == null; - } - - private static boolean areContiguousTablets(KeyExtent firstTablet, KeyExtent lastTablet) { - return firstTablet != null && lastTablet != null - && Objects.equals(firstTablet.endRow(), lastTablet.prevEndRow()); - } - - private boolean hasTabletsToDelete(final KeyExtent firstTabletInRange, - final KeyExtent lastTableInRange) { - // If the tablets are equal (and not null) then the deletion range is just part of 1 tablet - // which will be fenced so there are no tablets to delete. The null check is because if both - // are null then we are just deleting everything, so we do have tablets to delete - if (Objects.equals(firstTabletInRange, lastTableInRange) && firstTabletInRange != null) { - Manager.log.trace( - "No tablets to delete, firstTablet {} equals lastTablet {} in deletion range and was fenced.", - firstTabletInRange, lastTableInRange); - return false; - // If the lastTablet of the deletion range is the first tablet of the table it has been fenced - // already so nothing to actually delete before it - } else if (isFirstTabletInTable(lastTableInRange)) { - Manager.log.trace( - "No tablets to delete, lastTablet {} in deletion range is the first tablet of the table and was fenced.", - lastTableInRange); - return false; - // If the firstTablet of the deletion range is the last tablet of the table it has been fenced - // already so nothing to actually delete after it - } else if (isLastTabletInTable(firstTabletInRange)) { - Manager.log.trace( - "No tablets to delete, firstTablet {} in deletion range is the last tablet of the table and was fenced.", - firstTabletInRange); - return false; - // If the firstTablet and lastTablet are contiguous tablets then there is nothing to delete as - // each will be fenced and nothing between - } else if (areContiguousTablets(firstTabletInRange, lastTableInRange)) { - Manager.log.trace( - "No tablets to delete, firstTablet {} and lastTablet {} in deletion range are contiguous and were fenced.", - firstTabletInRange, lastTableInRange); - return false; - } - - return true; - } - - private void deleteTablets(MergeInfo info) throws AccumuloException { - // Before updated metadata and get the first and last tablets which - // are fenced if necessary - final Pair firstAndLastTablets = updateMetadataRecordsForDelete(info); - - // Find the deletion start row (exclusive) for tablets that need to be actually deleted - // This will be null if deleting everything up until the end row or it will be - // the endRow of the first tablet as the first tablet should be kept and will have - // already been fenced if necessary - final Text deletionStartRow = getDeletionStartRow(firstAndLastTablets.getFirst()); - - // Find the deletion end row (inclusive) for tablets that need to be actually deleted - // This will be null if deleting everything after the starting row or it will be - // the prevEndRow of the last tablet as the last tablet should be kept and will have - // already been fenced if necessary - Text deletionEndRow = getDeletionEndRow(firstAndLastTablets.getSecond()); - - // check if there are any tablets to delete and if not return - if (!hasTabletsToDelete(firstAndLastTablets.getFirst(), firstAndLastTablets.getSecond())) { - Manager.log.trace("No tablets to delete for range {}, returning", info.getExtent()); - return; - } - - // Build an extent for the actual deletion range - final KeyExtent extent = - new KeyExtent(info.getExtent().tableId(), deletionEndRow, deletionStartRow); - Manager.log.debug("Tablet deletion range is {}", extent); - String targetSystemTable = extent.isMeta() ? RootTable.NAME : MetadataTable.NAME; - Manager.log.debug("Deleting tablets for {}", extent); - MetadataTime metadataTime = null; - KeyExtent followingTablet = null; - Set goals = new HashSet<>(); - if (extent.endRow() != null) { - Key nextExtent = new Key(extent.endRow()).followingKey(PartialKey.ROW); - followingTablet = - getHighTablet(new KeyExtent(extent.tableId(), nextExtent.getRow(), extent.endRow())); - Manager.log.debug("Found following tablet {}", followingTablet); - } - try { - AccumuloClient client = manager.getContext(); - ServerContext context = manager.getContext(); - Ample ample = context.getAmple(); - Text start = extent.prevEndRow(); - if (start == null) { - start = new Text(); - } - Manager.log.debug("Making file deletion entries for {}", extent); - Range deleteRange = new Range(TabletsSection.encodeRow(extent.tableId(), start), false, - TabletsSection.encodeRow(extent.tableId(), extent.endRow()), true); - Scanner scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); - scanner.setRange(deleteRange); - ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner); - ServerColumnFamily.TIME_COLUMN.fetch(scanner); - HostingColumnFamily.GOAL_COLUMN.fetch(scanner); - scanner.fetchColumnFamily(DataFileColumnFamily.NAME); - scanner.fetchColumnFamily(CurrentLocationColumnFamily.NAME); - Set datafilesAndDirs = new TreeSet<>(); - for (Entry entry : scanner) { - Key key = entry.getKey(); - if (key.compareColumnFamily(DataFileColumnFamily.NAME) == 0) { - var stf = new StoredTabletFile(key.getColumnQualifierData().toString()); - datafilesAndDirs.add(new ReferenceFile(stf.getTableId(), stf)); - if (datafilesAndDirs.size() > 1000) { - ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); - datafilesAndDirs.clear(); - } - } else if (ServerColumnFamily.TIME_COLUMN.hasColumns(key)) { - metadataTime = MetadataTime.parse(entry.getValue().toString()); - } else if (key.compareColumnFamily(CurrentLocationColumnFamily.NAME) == 0) { - throw new IllegalStateException( - "Tablet " + key.getRow() + " is assigned during a merge!"); - } else if (ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) { - var allVolumesDirectory = - new AllVolumesDirectory(extent.tableId(), entry.getValue().toString()); - datafilesAndDirs.add(allVolumesDirectory); - if (datafilesAndDirs.size() > 1000) { - ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); - datafilesAndDirs.clear(); - } - } else if (HostingColumnFamily.GOAL_COLUMN.hasColumns(key)) { - TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(entry.getValue()); - goals.add(thisGoal); - } - } - ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); - BatchWriter bw = client.createBatchWriter(targetSystemTable); - try { - deleteTablets(info, deleteRange, bw, client); - } finally { - bw.close(); - } - - if (followingTablet != null) { - Manager.log.debug("Updating prevRow of {} to {}", followingTablet, extent.prevEndRow()); - bw = client.createBatchWriter(targetSystemTable); - try { - Mutation m = new Mutation(followingTablet.toMetaRow()); - TabletColumnFamily.PREV_ROW_COLUMN.put(m, - TabletColumnFamily.encodePrevEndRow(extent.prevEndRow())); - bw.addMutation(m); - bw.flush(); - } finally { - bw.close(); - } - } else { - // Recreate the default tablet to hold the end of the table - MetadataTableUtil.addTablet(new KeyExtent(extent.tableId(), null, extent.prevEndRow()), - ServerColumnFamily.DEFAULT_TABLET_DIR_NAME, manager.getContext(), - metadataTime.getType(), manager.managerLock, getMergeHostingGoal(extent, goals)); - } - } catch (RuntimeException | TableNotFoundException ex) { - throw new AccumuloException(ex); - } - } - - private void mergeMetadataRecords(MergeInfo info) throws AccumuloException { - KeyExtent range = info.getExtent(); - Manager.log.debug("Merging metadata for {}", range); - KeyExtent stop = getHighTablet(range); - Manager.log.debug("Highest tablet is {}", stop); - Value firstPrevRowValue = null; - Text stopRow = stop.toMetaRow(); - Text start = range.prevEndRow(); - if (start == null) { - start = new Text(); - } - Range scanRange = - new Range(TabletsSection.encodeRow(range.tableId(), start), false, stopRow, false); - String targetSystemTable = MetadataTable.NAME; - if (range.isMeta()) { - targetSystemTable = RootTable.NAME; - } - Set goals = new HashSet<>(); - - AccumuloClient client = manager.getContext(); - - KeyExtent stopExtent = KeyExtent.fromMetaRow(stop.toMetaRow()); - KeyExtent previousKeyExtent = null; - KeyExtent lastExtent = null; - - try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) { - long fileCount = 0; - // Make file entries in highest tablet - Scanner scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); - // Update to set the range to include the highest tablet - scanner.setRange( - new Range(TabletsSection.encodeRow(range.tableId(), start), false, stopRow, true)); - TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner); - ServerColumnFamily.TIME_COLUMN.fetch(scanner); - ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner); - HostingColumnFamily.GOAL_COLUMN.fetch(scanner); - scanner.fetchColumnFamily(DataFileColumnFamily.NAME); - Mutation m = new Mutation(stopRow); - MetadataTime maxLogicalTime = null; - for (Entry entry : scanner) { - Key key = entry.getKey(); - Value value = entry.getValue(); - - final KeyExtent keyExtent = KeyExtent.fromMetaRow(key.getRow()); - - // Keep track of the last Key Extent seen so we can use it to fence - // of RFiles when merging the metadata - if (lastExtent != null && !keyExtent.equals(lastExtent)) { - previousKeyExtent = lastExtent; - } - - // Special case to handle the highest/stop tablet, which is where files are - // merged to. The existing merge code won't delete files from this tablet - // so we need to handle the deletes in this tablet when fencing files. - // We may be able to make this simpler in the future. - if (keyExtent.equals(stopExtent)) { - if (previousKeyExtent != null - && key.getColumnFamily().equals(DataFileColumnFamily.NAME)) { - - // Fence off existing files by the end row of the previous tablet and current tablet - final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier()); - // The end row should be inclusive for the current tablet and the previous end row - // should be exclusive for the start row - Range fenced = new Range(previousKeyExtent.endRow(), false, keyExtent.endRow(), true); - - // Clip range if exists - fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced; - - final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); - // If the existing metadata does not match then we need to delete the old - // and replace with a new range - if (!existing.equals(newFile)) { - m.putDelete(DataFileColumnFamily.NAME, existing.getMetadataText()); - m.put(key.getColumnFamily(), newFile.getMetadataText(), value); - } - - fileCount++; - } - // For the highest tablet we only care about the DataFileColumnFamily - continue; - } - - // Handle metadata updates for all other tablets except the highest tablet - // Ranges are created for the files and then added to the highest tablet in - // the merge range. Deletes are handled later for the old files when the tablets - // are removed. - if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) { - final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier()); - - // Fence off files by the previous tablet and current tablet that is being merged - // The end row should be inclusive for the current tablet and the previous end row should - // be exclusive for the start row. - Range fenced = new Range(previousKeyExtent != null ? previousKeyExtent.endRow() : null, - false, keyExtent.endRow(), true); - - // Clip range with the tablet range if the range already exists - fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced; - - // Move the file and range to the last tablet - StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); - m.put(key.getColumnFamily(), newFile.getMetadataText(), value); - - fileCount++; - } else if (TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key) - && firstPrevRowValue == null) { - Manager.log.debug("prevRow entry for lowest tablet is {}", value); - firstPrevRowValue = new Value(value); - } else if (ServerColumnFamily.TIME_COLUMN.hasColumns(key)) { - maxLogicalTime = - TabletTime.maxMetadataTime(maxLogicalTime, MetadataTime.parse(value.toString())); - } else if (ServerColumnFamily.DIRECTORY_COLUMN.hasColumns(key)) { - var allVolumesDir = new AllVolumesDirectory(range.tableId(), value.toString()); - bw.addMutation(manager.getContext().getAmple().createDeleteMutation(allVolumesDir)); - } else if (HostingColumnFamily.GOAL_COLUMN.hasColumns(key)) { - TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(value); - goals.add(thisGoal); - } - - lastExtent = keyExtent; - } - - // read the logical time from the last tablet in the merge range, it is not included in - // the loop above - scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); - scanner.setRange(new Range(stopRow)); - ServerColumnFamily.TIME_COLUMN.fetch(scanner); - HostingColumnFamily.GOAL_COLUMN.fetch(scanner); - scanner.fetchColumnFamily(ExternalCompactionColumnFamily.NAME); - Set extCompIds = new HashSet<>(); - for (Entry entry : scanner) { - if (ServerColumnFamily.TIME_COLUMN.hasColumns(entry.getKey())) { - maxLogicalTime = TabletTime.maxMetadataTime(maxLogicalTime, - MetadataTime.parse(entry.getValue().toString())); - } else if (ExternalCompactionColumnFamily.NAME.equals(entry.getKey().getColumnFamily())) { - extCompIds.add(entry.getKey().getColumnQualifierData().toString()); - } else if (HostingColumnFamily.GOAL_COLUMN.hasColumns(entry.getKey())) { - TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(entry.getValue()); - goals.add(thisGoal); - } - } - - if (maxLogicalTime != null) { - ServerColumnFamily.TIME_COLUMN.put(m, new Value(maxLogicalTime.encode())); - } - - // delete any entries for external compactions - extCompIds.forEach(ecid -> m.putDelete(ExternalCompactionColumnFamily.STR_NAME, ecid)); - - // Set the TabletHostingGoal for this tablet based on the goals of the other tablets in - // the merge range. Always takes priority over never. - TabletHostingGoal mergeHostingGoal = getMergeHostingGoal(range, goals); - HostingColumnFamily.GOAL_COLUMN.put(m, TabletHostingGoalUtil.toValue(mergeHostingGoal)); - - if (!m.getUpdates().isEmpty()) { - bw.addMutation(m); - } - - bw.flush(); - - Manager.log.debug("Moved {} files to {}", fileCount, stop); - - if (firstPrevRowValue == null) { - Manager.log.debug("tablet already merged"); - return; - } - - stop = new KeyExtent(stop.tableId(), stop.endRow(), - TabletColumnFamily.decodePrevEndRow(firstPrevRowValue)); - Mutation updatePrevRow = TabletColumnFamily.createPrevRowMutation(stop); - Manager.log.debug("Setting the prevRow for last tablet: {}", stop); - bw.addMutation(updatePrevRow); - bw.flush(); - - deleteTablets(info, scanRange, bw, client); - - } catch (Exception ex) { - throw new AccumuloException(ex); - } - } - - private static TabletHostingGoal getMergeHostingGoal(KeyExtent range, - Set goals) { - TabletHostingGoal mergeHostingGoal = TabletHostingGoal.ONDEMAND; - if (range.isMeta() || goals.contains(TabletHostingGoal.ALWAYS)) { - mergeHostingGoal = TabletHostingGoal.ALWAYS; - } else if (goals.contains(TabletHostingGoal.NEVER)) { - mergeHostingGoal = TabletHostingGoal.NEVER; - } - return mergeHostingGoal; - } - - // This method is used to detect if a tablet needs to be split/chopped for a delete - // Instead of performing a split or chop compaction, the tablet will have its files fenced. - private boolean needsFencingForDeletion(MergeInfo info, KeyExtent keyExtent) { - // Does this extent cover the end points of the delete? - final Predicate isWithin = r -> r != null && keyExtent.contains(r); - final Predicate isNotBoundary = - r -> !r.equals(keyExtent.endRow()) && !r.equals(keyExtent.prevEndRow()); - final KeyExtent deleteRange = info.getExtent(); - - return (keyExtent.overlaps(deleteRange) && Stream - .of(deleteRange.prevEndRow(), deleteRange.endRow()).anyMatch(isWithin.and(isNotBoundary))) - || info.needsToBeChopped(keyExtent); - } - - // Instead of splitting or chopping tablets for a delete we instead create ranges - // to exclude the portion of the tablet that should be deleted - private Text followingRow(Text row) { - if (row == null) { - return null; - } - return new Key(row).followingKey(PartialKey.ROW).getRow(); - } - - // Instead of splitting or chopping tablets for a delete we instead create ranges - // to exclude the portion of the tablet that should be deleted - private List createRangesForDeletion(TabletMetadata tabletMetadata, - final KeyExtent deleteRange) { - final KeyExtent tabletExtent = tabletMetadata.getExtent(); - - // If the delete range wholly contains the tablet being deleted then there is no range to clip - // files to because the files should be completely dropped. - Preconditions.checkArgument(!deleteRange.contains(tabletExtent), "delete range:%s tablet:%s", - deleteRange, tabletExtent); - - final List ranges = new ArrayList<>(); - - if (deleteRange.overlaps(tabletExtent)) { - if (deleteRange.prevEndRow() != null - && tabletExtent.contains(followingRow(deleteRange.prevEndRow()))) { - Manager.log.trace("Fencing tablet {} files to ({},{}]", tabletExtent, - tabletExtent.prevEndRow(), deleteRange.prevEndRow()); - ranges.add(new Range(tabletExtent.prevEndRow(), false, deleteRange.prevEndRow(), true)); - } - - // This covers the case of when a deletion range overlaps the last tablet. We need to create a - // range that excludes the deletion. - if (deleteRange.endRow() != null - && tabletMetadata.getExtent().contains(deleteRange.endRow())) { - Manager.log.trace("Fencing tablet {} files to ({},{}]", tabletExtent, deleteRange.endRow(), - tabletExtent.endRow()); - ranges.add(new Range(deleteRange.endRow(), false, tabletExtent.endRow(), true)); - } - } else { - Manager.log.trace( - "Fencing tablet {} files to itself because it does not overlap delete range", - tabletExtent); - ranges.add(tabletExtent.toDataRange()); - } - - return ranges; - } - - private Pair updateMetadataRecordsForDelete(MergeInfo info) - throws AccumuloException { - final KeyExtent range = info.getExtent(); - - String targetSystemTable = MetadataTable.NAME; - if (range.isMeta()) { - targetSystemTable = RootTable.NAME; - } - final Pair startAndEndTablets; - - final AccumuloClient client = manager.getContext(); - - try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) { - final Text startRow = range.prevEndRow(); - final Text endRow = range.endRow() != null - ? new Key(range.endRow()).followingKey(PartialKey.ROW).getRow() : null; - - // Find the tablets that overlap the start and end row of the deletion range - // If the startRow is null then there will be an empty startTablet we don't need - // to fence a starting tablet as we are deleting everything up to the end tablet - // Likewise, if the endRow is null there will be an empty endTablet as we are deleting - // all tablets after the starting tablet - final Optional startTablet = Optional.ofNullable(startRow).flatMap( - row -> loadTabletMetadata(range.tableId(), row, ColumnType.PREV_ROW, ColumnType.FILES)); - final Optional endTablet = Optional.ofNullable(endRow).flatMap( - row -> loadTabletMetadata(range.tableId(), row, ColumnType.PREV_ROW, ColumnType.FILES)); - - // Store the tablets in a Map if present so that if we have the same Tablet we - // only need to process the same tablet once when fencing - final SortedMap tabletMetadatas = new TreeMap<>(); - startTablet.ifPresent(ft -> tabletMetadatas.put(ft.getExtent(), ft)); - endTablet.ifPresent(lt -> tabletMetadatas.putIfAbsent(lt.getExtent(), lt)); - - // Capture the tablets to return them or null if not loaded - startAndEndTablets = new Pair<>(startTablet.map(TabletMetadata::getExtent).orElse(null), - endTablet.map(TabletMetadata::getExtent).orElse(null)); - - for (TabletMetadata tabletMetadata : tabletMetadatas.values()) { - final KeyExtent keyExtent = tabletMetadata.getExtent(); - - // Check if this tablet needs to have its files fenced for the deletion - if (needsFencingForDeletion(info, keyExtent)) { - Manager.log.debug("Found overlapping keyExtent {} for delete, fencing files.", keyExtent); - - // Create the ranges for fencing the files, this takes the place of - // chop compactions and splits - final List ranges = createRangesForDeletion(tabletMetadata, range); - Preconditions.checkState(!ranges.isEmpty(), - "No ranges found that overlap deletion range."); - - // Go through and fence each of the files that are part of the tablet - for (Entry entry : tabletMetadata.getFilesMap() - .entrySet()) { - final StoredTabletFile existing = entry.getKey(); - final DataFileValue value = entry.getValue(); - - final Mutation m = new Mutation(keyExtent.toMetaRow()); - - // Go through each range that was created and modify the metadata for the file - // The end row should be inclusive for the current tablet and the previous end row - // should be exclusive for the start row. - final Set newFiles = new HashSet<>(); - final Set existingFile = Set.of(existing); - - for (Range fenced : ranges) { - // Clip range with the tablet range if the range already exists - fenced = existing.hasRange() ? existing.getRange().clip(fenced, true) : fenced; - - // If null the range is disjoint which can happen if there are existing fenced files - // If the existing file is disjoint then later we will delete if the file is not part - // of the newFiles set which means it is disjoint with all ranges - if (fenced != null) { - final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); - Manager.log.trace("Adding new file {} with range {}", newFile.getMetadataPath(), - newFile.getRange()); - - // Add the new file to the newFiles set, it will be added later if it doesn't match - // the existing file already. We still need to add to the set to be checked later - // even if it matches the existing file as later the deletion logic will check to - // see if the existing file is part of this set before deleting. This is done to - // make sure the existing file isn't deleted unless it is not needed/disjoint - // with all ranges. - newFiles.add(newFile); - } else { - Manager.log.trace("Found a disjoint file {} with range {} on delete", - existing.getMetadataPath(), existing.getRange()); - } - } - - // If the existingFile is not contained in the newFiles set then we can delete it - Sets.difference(existingFile, newFiles).forEach( - delete -> m.putDelete(DataFileColumnFamily.NAME, existing.getMetadataText())); - - // Add any new files that don't match the existingFile - // As of now we will only have at most 2 files as up to 2 ranges are created - final List filesToAdd = - new ArrayList<>(Sets.difference(newFiles, existingFile)); - Preconditions.checkArgument(filesToAdd.size() <= 2, - "There should only be at most 2 StoredTabletFiles after computing new ranges."); - - // If more than 1 new file then re-calculate the num entries and size - if (filesToAdd.size() == 2) { - // This splits up the values in half and makes sure they total the original - // values - final Pair newDfvs = computeNewDfv(value); - m.put(DataFileColumnFamily.NAME, filesToAdd.get(0).getMetadataText(), - newDfvs.getFirst().encodeAsValue()); - m.put(DataFileColumnFamily.NAME, filesToAdd.get(1).getMetadataText(), - newDfvs.getSecond().encodeAsValue()); - } else { - // Will be 0 or 1 files - filesToAdd.forEach(newFile -> m.put(DataFileColumnFamily.NAME, - newFile.getMetadataText(), value.encodeAsValue())); - } - - if (!m.getUpdates().isEmpty()) { - bw.addMutation(m); - } - } - } else { - Manager.log.debug( - "Skipping metadata update on file for keyExtent {} for delete as not overlapping on rows.", - keyExtent); - } - } - - bw.flush(); - - return startAndEndTablets; - } catch (Exception ex) { - throw new AccumuloException(ex); - } - } - - // Divide each new DFV in half and make sure the sum equals the original - @VisibleForTesting - protected static Pair computeNewDfv(DataFileValue value) { - final DataFileValue file1Value = new DataFileValue(Math.max(1, value.getSize() / 2), - Math.max(1, value.getNumEntries() / 2), value.getTime()); - - final DataFileValue file2Value = - new DataFileValue(Math.max(1, value.getSize() - file1Value.getSize()), - Math.max(1, value.getNumEntries() - file1Value.getNumEntries()), value.getTime()); - - return new Pair<>(file1Value, file2Value); - } - - private Optional loadTabletMetadata(TableId tabletId, final Text row, - ColumnType... columns) { - try (TabletsMetadata tabletsMetadata = manager.getContext().getAmple().readTablets() - .forTable(tabletId).overlapping(row, true, row).fetch(columns).build()) { - return tabletsMetadata.stream().findFirst(); - } - } - - private void deleteTablets(MergeInfo info, Range scanRange, BatchWriter bw, AccumuloClient client) - throws TableNotFoundException, MutationsRejectedException { - Scanner scanner; - Mutation m; - // Delete everything in the other tablets - // group all deletes into tablet into one mutation, this makes tablets - // either disappear entirely or not all.. this is important for the case - // where the process terminates in the loop below... - scanner = client.createScanner(info.getExtent().isMeta() ? RootTable.NAME : MetadataTable.NAME, - Authorizations.EMPTY); - Manager.log.debug("Deleting range {}", scanRange); - scanner.setRange(scanRange); - RowIterator rowIter = new RowIterator(scanner); - while (rowIter.hasNext()) { - Iterator> row = rowIter.next(); - m = null; - while (row.hasNext()) { - Entry entry = row.next(); - Key key = entry.getKey(); - - if (m == null) { - m = new Mutation(key.getRow()); - } - - m.putDelete(key.getColumnFamily(), key.getColumnQualifier()); - Manager.log.debug("deleting entry {}", key); - } - bw.addMutation(m); - } - - bw.flush(); - } - - private KeyExtent getHighTablet(KeyExtent range) throws AccumuloException { - try { - AccumuloClient client = manager.getContext(); - Scanner scanner = client.createScanner(range.isMeta() ? RootTable.NAME : MetadataTable.NAME, - Authorizations.EMPTY); - TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner); - KeyExtent start = new KeyExtent(range.tableId(), range.endRow(), null); - scanner.setRange(new Range(start.toMetaRow(), null)); - Iterator> iterator = scanner.iterator(); - if (!iterator.hasNext()) { - throw new AccumuloException("No last tablet for a merge " + range); - } - Entry entry = iterator.next(); - KeyExtent highTablet = KeyExtent.fromMetaPrevRow(entry); - if (!highTablet.tableId().equals(range.tableId())) { - throw new AccumuloException("No last tablet for merge " + range + " " + highTablet); - } - return highTablet; - } catch (Exception ex) { - throw new AccumuloException("Unexpected failure finding the last tablet for a merge " + range, - ex); - } - } - private void handleDeadTablets(TabletLists tLists) throws WalMarkerException, DistributedStoreException { var deadTablets = tLists.assignedToDeadServers; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java index eb2cf605fdf..329f57b24ad 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/compaction/coordinator/CompactionCoordinator.java @@ -563,7 +563,7 @@ private ExternalCompactionMetadata reserveCompaction(CompactionJobQueues.MetaJob // any data that is read from the tablet to make a decision about if it can compact or not // must be included in the requireSame call var tabletMutator = tabletsMutator.mutateTablet(extent).requireAbsentOperation() - .requireSame(tabletMetadata, PREV_ROW, FILES, SELECTED, ECOMP); + .requireSame(tabletMetadata, FILES, SELECTED, ECOMP); var ecid = ExternalCompactionId.of(externalCompactionId); tabletMutator.putExternalCompaction(ecid, ecm); @@ -926,7 +926,7 @@ private TabletMetadata commitCompaction(TCompactionStats stats, ExternalCompacti try (var tabletsMutator = ctx.getAmple().conditionallyMutateTablets()) { var tabletMutator = tabletsMutator.mutateTablet(extent).requireAbsentOperation() - .requireCompaction(ecid).requireSame(tablet, PREV_ROW, FILES, LOCATION); + .requireCompaction(ecid).requireSame(tablet, FILES, LOCATION); if (ecm.getKind() == CompactionKind.USER || ecm.getKind() == CompactionKind.SELECTOR) { tabletMutator.requireSame(tablet, SELECTED, COMPACTED); @@ -1050,7 +1050,7 @@ void compactionFailed(Map compactions) { try { ctx.requireNotDeleted(extent.tableId()); tabletsMutator.mutateTablet(extent).requireAbsentOperation().requireCompaction(ecid) - .requirePrevEndRow(extent.prevEndRow()).deleteExternalCompaction(ecid) + .deleteExternalCompaction(ecid) .submit(tabletMetadata -> !tabletMetadata.getExternalCompactions().containsKey(ecid)); } catch (TableDeletedException e) { LOG.warn("Table {} was deleted, unable to update metadata for compaction failure.", diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java b/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java deleted file mode 100644 index 59e4b087f1b..00000000000 --- a/server/manager/src/main/java/org/apache/accumulo/manager/state/MergeStats.java +++ /dev/null @@ -1,238 +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.manager.state; - -import java.io.IOException; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Set; - -import org.apache.accumulo.core.Constants; -import org.apache.accumulo.core.client.Accumulo; -import org.apache.accumulo.core.client.AccumuloClient; -import org.apache.accumulo.core.client.Scanner; -import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.fate.zookeeper.ZooReaderWriter; -import org.apache.accumulo.core.fate.zookeeper.ZooUtil; -import org.apache.accumulo.core.manager.state.TabletManagement; -import org.apache.accumulo.core.metadata.MetadataTable; -import org.apache.accumulo.core.metadata.RootTable; -import org.apache.accumulo.core.metadata.TServerInstance; -import org.apache.accumulo.core.metadata.TabletState; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection; -import org.apache.accumulo.core.metadata.schema.TabletMetadata; -import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.trace.TraceUtil; -import org.apache.accumulo.server.cli.ServerUtilOpts; -import org.apache.accumulo.server.manager.state.CurrentState; -import org.apache.accumulo.server.manager.state.MergeInfo; -import org.apache.accumulo.server.manager.state.MergeState; -import org.apache.accumulo.server.manager.state.TabletManagementIterator; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.Text; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.opentelemetry.api.trace.Span; -import io.opentelemetry.context.Scope; - -public class MergeStats { - final static private Logger log = LoggerFactory.getLogger(MergeStats.class); - private final MergeInfo info; - private int hosted = 0; - private int unassigned = 0; - private int total = 0; - - public MergeStats(MergeInfo info) { - this.info = info; - } - - public MergeInfo getMergeInfo() { - return info; - } - - public void update(KeyExtent ke, TabletState state) { - if (info.getState().equals(MergeState.NONE)) { - return; - } - if (!info.overlaps(ke)) { - return; - } - this.total++; - if (state.equals(TabletState.HOSTED)) { - this.hosted++; - } - if (state.equals(TabletState.UNASSIGNED) || state.equals(TabletState.SUSPENDED)) { - this.unassigned++; - } - } - - public MergeState nextMergeState(AccumuloClient accumuloClient, CurrentState manager) - throws Exception { - MergeState state = info.getState(); - if (state == MergeState.NONE) { - return state; - } - if (total == 0) { - log.trace("failed to see any tablets for this range, ignoring {}", info.getExtent()); - return state; - } - log.info("Computing next merge state for {} which is presently {} isDelete : {}", - info.getExtent(), state, info.isDelete()); - if (state == MergeState.STARTED) { - log.info("{} are hosted, total {}", hosted, total); - if (!info.isDelete() && total == 1) { - log.info("Merge range is already contained in a single tablet {}", info.getExtent()); - state = MergeState.COMPLETE; - } else if (hosted == total) { - state = MergeState.WAITING_FOR_OFFLINE; - } else { - log.info("Waiting for {} hosted tablets to be {} {}", hosted, total, info.getExtent()); - } - } - if (state == MergeState.WAITING_FOR_OFFLINE) { - if (unassigned == total) { - if (verifyMergeConsistency(accumuloClient, manager)) { - state = MergeState.MERGING; - } else { - log.info("Merge consistency check failed {}", info.getExtent()); - } - } else { - log.info("Waiting for {} unassigned tablets to be {} {}", unassigned, total, - info.getExtent()); - } - } - if (state == MergeState.MERGING) { - if (hosted != 0) { - // Shouldn't happen - log.error("Unexpected state: hosted tablets should be zero {} merge {}", hosted, - info.getExtent()); - state = MergeState.WAITING_FOR_OFFLINE; - } - if (unassigned != total) { - // Shouldn't happen - log.error("Unexpected state: unassigned tablets should be {} was {} merge {}", total, - unassigned, info.getExtent()); - state = MergeState.WAITING_FOR_OFFLINE; - } - log.info("{} tablets are unassigned {}", unassigned, info.getExtent()); - } - return state; - } - - private boolean verifyMergeConsistency(AccumuloClient accumuloClient, CurrentState manager) - throws TableNotFoundException, IOException { - MergeStats verify = new MergeStats(info); - KeyExtent extent = info.getExtent(); - Scanner scanner = accumuloClient - .createScanner(extent.isMeta() ? RootTable.NAME : MetadataTable.NAME, Authorizations.EMPTY); - TabletManagementIterator.configureScanner(scanner, manager); - Text start = extent.prevEndRow(); - if (start == null) { - start = new Text(); - } - TableId tableId = extent.tableId(); - Text first = TabletsSection.encodeRow(tableId, start); - Range range = new Range(first, false, null, true); - scanner.setRange(range.clip(TabletsSection.getRange())); - KeyExtent prevExtent = null; - - log.debug("Scanning range {}", range); - for (Entry entry : scanner) { - final TabletManagement mti = TabletManagementIterator.decode(entry); - final TabletMetadata tm = mti.getTabletMetadata(); - - log.debug("consistency check: {} walogs {}", tm, tm.getLogs().size()); - if (!tm.getTableId().equals(tableId)) { - break; - } - - if (prevExtent == null) { - // this is the first tablet observed, it must be offline and its prev row must be less than - // the start of the merge range - if (tm.getExtent().prevEndRow() != null - && tm.getExtent().prevEndRow().compareTo(start) > 0) { - log.debug("failing consistency: prev row is too high {}", start); - return false; - } - - Set liveTServers1 = manager.onlineTabletServers(); - if (TabletState.compute(tm, liveTServers1) != TabletState.UNASSIGNED) { - Set liveTServers = manager.onlineTabletServers(); - if (TabletState.compute(tm, liveTServers) != TabletState.SUSPENDED) { - log.debug("failing consistency: assigned or hosted {}", tm); - return false; - } - } - - } else if (!tm.getExtent().isPreviousExtent(prevExtent)) { - log.debug("hole in {}", MetadataTable.NAME); - return false; - } - - prevExtent = tm.getExtent(); - - Set liveTServers = manager.onlineTabletServers(); - verify.update(tm.getExtent(), TabletState.compute(tm, liveTServers)); - // stop when we've seen the tablet just beyond our range - if (tm.getExtent().prevEndRow() != null && extent.endRow() != null - && tm.getExtent().prevEndRow().compareTo(extent.endRow()) > 0) { - break; - } - } - log.debug("unassigned {} v.unassigned {} verify.total {}", unassigned, verify.unassigned, - verify.total); - - return unassigned == verify.unassigned && unassigned == verify.total; - } - - public static void main(String[] args) throws Exception { - ServerUtilOpts opts = new ServerUtilOpts(); - opts.parseArgs(MergeStats.class.getName(), args); - - Span span = TraceUtil.startSpan(MergeStats.class, "main"); - try (Scope scope = span.makeCurrent()) { - try (AccumuloClient client = Accumulo.newClient().from(opts.getClientProps()).build()) { - Map tableIdMap = client.tableOperations().tableIdMap(); - ZooReaderWriter zooReaderWriter = opts.getServerContext().getZooReaderWriter(); - for (Entry entry : tableIdMap.entrySet()) { - final String table = entry.getKey(), tableId = entry.getValue(); - String path = ZooUtil.getRoot(client.instanceOperations().getInstanceId()) - + Constants.ZTABLES + "/" + tableId + "/merge"; - MergeInfo info = new MergeInfo(); - if (zooReaderWriter.exists(path)) { - byte[] data = zooReaderWriter.getData(path); - DataInputBuffer in = new DataInputBuffer(); - in.reset(data, data.length); - info.readFields(in); - } - System.out.printf("%25s %10s %10s %s%n", table, info.getState(), info.getOperation(), - info.getExtent()); - } - } - } finally { - span.end(); - } - } -} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java index 60738c04759..15109b9a443 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/bulkVer2/LoadFiles.java @@ -144,9 +144,9 @@ void load(List tablets, Files files) { conditionalMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation(); if (setTime) { - tabletMutator.requireSame(tablet, PREV_ROW, LOADED, TIME, LOCATION); + tabletMutator.requireSame(tablet, LOADED, TIME, LOCATION); } else { - tabletMutator.requireSame(tablet, PREV_ROW, LOADED); + tabletMutator.requireSame(tablet, LOADED); } filesToLoad.forEach((f, v) -> { diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java index cef3b05dca5..9b864e3f3fd 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CleanUp.java @@ -68,7 +68,7 @@ public long isReady(long tid, Manager manager) throws Exception { for (TabletMetadata tablet : tablets) { if (tablet.getCompacted().contains(tid)) { tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation() - .requireSame(tablet, PREV_ROW, COMPACTED).deleteCompacted(tid) + .requireSame(tablet, COMPACTED).deleteCompacted(tid) .submit(tabletMetadata -> !tabletMetadata.getCompacted().contains(tid)); } } diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java index 7868aee84c7..21d0f059025 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/compact/CompactionDriver.java @@ -171,7 +171,7 @@ public int updateAndCheckTablets(Manager manager, long tid) FateTxId.formatTid(tid), tablet.getExtent()); // this tablet has no files try to mark it as done tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation() - .requireSame(tablet, PREV_ROW, FILES, COMPACTED).putCompacted(tid) + .requireSame(tablet, FILES, COMPACTED).putCompacted(tid) .submit(tabletMetadata -> tabletMetadata.getCompacted().contains(tid)); } else if (tablet.getSelectedFiles() == null && tablet.getExternalCompactions().isEmpty()) { // there are no selected files @@ -201,11 +201,11 @@ public int updateAndCheckTablets(Manager manager, long tid) if (filesToCompact.isEmpty()) { // no files were selected so mark the tablet as compacted tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation() - .requireSame(tablet, PREV_ROW, FILES, SELECTED, ECOMP, COMPACTED).putCompacted(tid) + .requireSame(tablet, FILES, SELECTED, ECOMP, COMPACTED).putCompacted(tid) .submit(tabletMetadata -> tabletMetadata.getCompacted().contains(tid)); } else { var mutator = tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation() - .requireSame(tablet, PREV_ROW, FILES, SELECTED, ECOMP, COMPACTED); + .requireSame(tablet, FILES, SELECTED, ECOMP, COMPACTED); var selectedFiles = new SelectedFiles(filesToCompact, tablet.getFiles().equals(filesToCompact), tid); @@ -309,7 +309,7 @@ private void cleanupTabletMetadata(long tid, Manager manager) throws Exception { if (needsUpdate.test(tablet)) { var mutator = tabletsMutator.mutateTablet(tablet.getExtent()).requireAbsentOperation() - .requireSame(tablet, PREV_ROW, COMPACTED, SELECTED); + .requireSame(tablet, COMPACTED, SELECTED); if (tablet.getSelectedFiles() != null && tablet.getSelectedFiles().getFateTxId() == tid) { mutator.deleteSelectedFiles(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteRows.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteRows.java new file mode 100644 index 00000000000..c6614cae5f1 --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/DeleteRows.java @@ -0,0 +1,600 @@ +/* + * 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.manager.tableOps.merge; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.SortedMap; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.function.Predicate; +import java.util.stream.Stream; + +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.MutationsRejectedException; +import org.apache.accumulo.core.client.RowIterator; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.TableNotFoundException; +import org.apache.accumulo.core.client.admin.TabletHostingGoal; +import org.apache.accumulo.core.clientImpl.TabletHostingGoalUtil; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.PartialKey; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.gc.ReferenceFile; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.Ample; +import org.apache.accumulo.core.metadata.schema.DataFileValue; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.metadata.schema.MetadataTime; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.core.metadata.schema.TabletsMetadata; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.util.Pair; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.apache.accumulo.server.ServerContext; +import org.apache.accumulo.server.gc.AllVolumesDirectory; +import org.apache.accumulo.server.manager.state.MergeInfo; +import org.apache.accumulo.server.manager.state.MergeState; +import org.apache.accumulo.server.util.MetadataTableUtil; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; + +public class DeleteRows extends ManagerRepo { + + private static final long serialVersionUID = 1L; + + private static final Logger log = LoggerFactory.getLogger(DeleteRows.class); + + private final NamespaceId namespaceId; + private final TableId tableId; + + public DeleteRows(NamespaceId namespaceId, TableId tableId) { + this.namespaceId = namespaceId; + this.tableId = tableId; + } + + @Override + public Repo call(long tid, Manager manager) throws Exception { + MergeInfo mergeInfo = manager.getMergeInfo(tableId); + Preconditions.checkState(mergeInfo.getState() == MergeState.MERGING); + Preconditions.checkState(mergeInfo.isDelete()); + + deleteTablets(manager, mergeInfo); + + manager.setMergeState(mergeInfo, MergeState.COMPLETE); + + // TODO namespace id + return new FinishTableRangeOp(namespaceId, tableId); + } + + private void deleteTablets(Manager manager, MergeInfo info) throws AccumuloException { + // Before updated metadata and get the first and last tablets which + // are fenced if necessary + final Pair firstAndLastTablets = + updateMetadataRecordsForDelete(manager, info); + + // Find the deletion start row (exclusive) for tablets that need to be actually deleted + // This will be null if deleting everything up until the end row or it will be + // the endRow of the first tablet as the first tablet should be kept and will have + // already been fenced if necessary + final Text deletionStartRow = getDeletionStartRow(firstAndLastTablets.getFirst()); + + // Find the deletion end row (inclusive) for tablets that need to be actually deleted + // This will be null if deleting everything after the starting row or it will be + // the prevEndRow of the last tablet as the last tablet should be kept and will have + // already been fenced if necessary + Text deletionEndRow = getDeletionEndRow(firstAndLastTablets.getSecond()); + + // check if there are any tablets to delete and if not return + if (!hasTabletsToDelete(firstAndLastTablets.getFirst(), firstAndLastTablets.getSecond())) { + log.trace("No tablets to delete for range {}, returning", info.getExtent()); + return; + } + + // Build an extent for the actual deletion range + final KeyExtent extent = + new KeyExtent(info.getExtent().tableId(), deletionEndRow, deletionStartRow); + log.debug("Tablet deletion range is {}", extent); + String targetSystemTable = extent.isMeta() ? RootTable.NAME : MetadataTable.NAME; + log.debug("Deleting tablets for {}", extent); + MetadataTime metadataTime = null; + KeyExtent followingTablet = null; + Set goals = new HashSet<>(); + if (extent.endRow() != null) { + Key nextExtent = new Key(extent.endRow()).followingKey(PartialKey.ROW); + followingTablet = getHighTablet(manager, + new KeyExtent(extent.tableId(), nextExtent.getRow(), extent.endRow())); + log.debug("Found following tablet {}", followingTablet); + } + try { + AccumuloClient client = manager.getContext(); + ServerContext context = manager.getContext(); + Ample ample = context.getAmple(); + Text start = extent.prevEndRow(); + if (start == null) { + start = new Text(); + } + log.debug("Making file deletion entries for {}", extent); + Range deleteRange = + new Range(MetadataSchema.TabletsSection.encodeRow(extent.tableId(), start), false, + MetadataSchema.TabletsSection.encodeRow(extent.tableId(), extent.endRow()), true); + Scanner scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); + scanner.setRange(deleteRange); + MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.fetch(scanner); + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME); + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME); + Set datafilesAndDirs = new TreeSet<>(); + for (Map.Entry entry : scanner) { + Key key = entry.getKey(); + if (key.compareColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME) == 0) { + var stf = new StoredTabletFile(key.getColumnQualifierData().toString()); + datafilesAndDirs.add(new ReferenceFile(stf.getTableId(), stf)); + if (datafilesAndDirs.size() > 1000) { + ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); + datafilesAndDirs.clear(); + } + } else if (MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(key)) { + metadataTime = MetadataTime.parse(entry.getValue().toString()); + } else if (key.compareColumnFamily( + MetadataSchema.TabletsSection.CurrentLocationColumnFamily.NAME) == 0) { + throw new IllegalStateException( + "Tablet " + key.getRow() + " is assigned during a merge!"); + } else if (MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN + .hasColumns(key)) { + var allVolumesDirectory = + new AllVolumesDirectory(extent.tableId(), entry.getValue().toString()); + datafilesAndDirs.add(allVolumesDirectory); + if (datafilesAndDirs.size() > 1000) { + ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); + datafilesAndDirs.clear(); + } + } else if (MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.hasColumns(key)) { + TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(entry.getValue()); + goals.add(thisGoal); + } + } + ample.putGcFileAndDirCandidates(extent.tableId(), datafilesAndDirs); + BatchWriter bw = client.createBatchWriter(targetSystemTable); + try { + deleteTablets(info, deleteRange, bw, client); + } finally { + bw.close(); + } + + if (followingTablet != null) { + log.debug("Updating prevRow of {} to {}", followingTablet, extent.prevEndRow()); + bw = client.createBatchWriter(targetSystemTable); + try { + Mutation m = new Mutation(followingTablet.toMetaRow()); + MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.put(m, + MetadataSchema.TabletsSection.TabletColumnFamily + .encodePrevEndRow(extent.prevEndRow())); + bw.addMutation(m); + bw.flush(); + } finally { + bw.close(); + } + } else { + // Recreate the default tablet to hold the end of the table + MetadataTableUtil.addTablet(new KeyExtent(extent.tableId(), null, extent.prevEndRow()), + MetadataSchema.TabletsSection.ServerColumnFamily.DEFAULT_TABLET_DIR_NAME, + manager.getContext(), metadataTime.getType(), manager.getManagerLock(), + getMergeHostingGoal(extent, goals)); + } + } catch (RuntimeException | TableNotFoundException ex) { + throw new AccumuloException(ex); + } + } + + private Pair updateMetadataRecordsForDelete(Manager manager, MergeInfo info) + throws AccumuloException { + final KeyExtent range = info.getExtent(); + + String targetSystemTable = MetadataTable.NAME; + if (range.isMeta()) { + targetSystemTable = RootTable.NAME; + } + final Pair startAndEndTablets; + + final AccumuloClient client = manager.getContext(); + + try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) { + final Text startRow = range.prevEndRow(); + final Text endRow = range.endRow() != null + ? new Key(range.endRow()).followingKey(PartialKey.ROW).getRow() : null; + + // Find the tablets that overlap the start and end row of the deletion range + // If the startRow is null then there will be an empty startTablet we don't need + // to fence a starting tablet as we are deleting everything up to the end tablet + // Likewise, if the endRow is null there will be an empty endTablet as we are deleting + // all tablets after the starting tablet + final Optional startTablet = + Optional.ofNullable(startRow).flatMap(row -> loadTabletMetadata(manager, range.tableId(), + row, TabletMetadata.ColumnType.PREV_ROW, TabletMetadata.ColumnType.FILES)); + final Optional endTablet = + Optional.ofNullable(endRow).flatMap(row -> loadTabletMetadata(manager, range.tableId(), + row, TabletMetadata.ColumnType.PREV_ROW, TabletMetadata.ColumnType.FILES)); + + // Store the tablets in a Map if present so that if we have the same Tablet we + // only need to process the same tablet once when fencing + final SortedMap tabletMetadatas = new TreeMap<>(); + startTablet.ifPresent(ft -> tabletMetadatas.put(ft.getExtent(), ft)); + endTablet.ifPresent(lt -> tabletMetadatas.putIfAbsent(lt.getExtent(), lt)); + + // Capture the tablets to return them or null if not loaded + startAndEndTablets = new Pair<>(startTablet.map(TabletMetadata::getExtent).orElse(null), + endTablet.map(TabletMetadata::getExtent).orElse(null)); + + for (TabletMetadata tabletMetadata : tabletMetadatas.values()) { + final KeyExtent keyExtent = tabletMetadata.getExtent(); + + // Check if this tablet needs to have its files fenced for the deletion + if (needsFencingForDeletion(info, keyExtent)) { + log.debug("Found overlapping keyExtent {} for delete, fencing files.", keyExtent); + + // Create the ranges for fencing the files, this takes the place of + // chop compactions and splits + final List ranges = createRangesForDeletion(tabletMetadata, range); + Preconditions.checkState(!ranges.isEmpty(), + "No ranges found that overlap deletion range."); + + // Go through and fence each of the files that are part of the tablet + for (Map.Entry entry : tabletMetadata.getFilesMap() + .entrySet()) { + final StoredTabletFile existing = entry.getKey(); + final DataFileValue value = entry.getValue(); + + final Mutation m = new Mutation(keyExtent.toMetaRow()); + + // Go through each range that was created and modify the metadata for the file + // The end row should be inclusive for the current tablet and the previous end row + // should be exclusive for the start row. + final Set newFiles = new HashSet<>(); + final Set existingFile = Set.of(existing); + + for (Range fenced : ranges) { + // Clip range with the tablet range if the range already exists + fenced = existing.hasRange() ? existing.getRange().clip(fenced, true) : fenced; + + // If null the range is disjoint which can happen if there are existing fenced files + // If the existing file is disjoint then later we will delete if the file is not part + // of the newFiles set which means it is disjoint with all ranges + if (fenced != null) { + final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); + log.trace("Adding new file {} with range {}", newFile.getMetadataPath(), + newFile.getRange()); + + // Add the new file to the newFiles set, it will be added later if it doesn't match + // the existing file already. We still need to add to the set to be checked later + // even if it matches the existing file as later the deletion logic will check to + // see if the existing file is part of this set before deleting. This is done to + // make sure the existing file isn't deleted unless it is not needed/disjoint + // with all ranges. + newFiles.add(newFile); + } else { + log.trace("Found a disjoint file {} with range {} on delete", + existing.getMetadataPath(), existing.getRange()); + } + } + + // If the existingFile is not contained in the newFiles set then we can delete it + Sets.difference(existingFile, newFiles).forEach( + delete -> m.putDelete(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, + existing.getMetadataText())); + + // Add any new files that don't match the existingFile + // As of now we will only have at most 2 files as up to 2 ranges are created + final List filesToAdd = + new ArrayList<>(Sets.difference(newFiles, existingFile)); + Preconditions.checkArgument(filesToAdd.size() <= 2, + "There should only be at most 2 StoredTabletFiles after computing new ranges."); + + // If more than 1 new file then re-calculate the num entries and size + if (filesToAdd.size() == 2) { + // This splits up the values in half and makes sure they total the original + // values + final Pair newDfvs = computeNewDfv(value); + m.put(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, + filesToAdd.get(0).getMetadataText(), newDfvs.getFirst().encodeAsValue()); + m.put(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, + filesToAdd.get(1).getMetadataText(), newDfvs.getSecond().encodeAsValue()); + } else { + // Will be 0 or 1 files + filesToAdd + .forEach(newFile -> m.put(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, + newFile.getMetadataText(), value.encodeAsValue())); + } + + if (!m.getUpdates().isEmpty()) { + bw.addMutation(m); + } + } + } else { + log.debug( + "Skipping metadata update on file for keyExtent {} for delete as not overlapping on rows.", + keyExtent); + } + } + + bw.flush(); + + return startAndEndTablets; + } catch (Exception ex) { + throw new AccumuloException(ex); + } + } + + // This method finds returns the deletion starting row (exclusive) for tablets that + // need to be actually deleted. If the startTablet is null then + // the deletion start row will just be null as all tablets are being deleted + // up to the end. Otherwise, this returns the endRow of the first tablet + // as the first tablet should be kept and will have been previously + // fenced if necessary + private Text getDeletionStartRow(final KeyExtent startTablet) { + if (startTablet == null) { + log.debug("First tablet for delete range is null"); + return null; + } + + final Text deletionStartRow = startTablet.endRow(); + log.debug("Start row is {} for deletion", deletionStartRow); + + return deletionStartRow; + } + + // This method finds returns the deletion ending row (inclusive) for tablets that + // need to be actually deleted. If the endTablet is null then + // the deletion end row will just be null as all tablets are being deleted + // after the start row. Otherwise, this returns the prevEndRow of the last tablet + // as the last tablet should be kept and will have been previously + // fenced if necessary + private Text getDeletionEndRow(final KeyExtent endTablet) { + if (endTablet == null) { + log.debug("Last tablet for delete range is null"); + return null; + } + + Text deletionEndRow = endTablet.prevEndRow(); + log.debug("Deletion end row is {}", deletionEndRow); + + return deletionEndRow; + } + + static TabletHostingGoal getMergeHostingGoal(KeyExtent range, Set goals) { + TabletHostingGoal mergeHostingGoal = TabletHostingGoal.ONDEMAND; + if (range.isMeta() || goals.contains(TabletHostingGoal.ALWAYS)) { + mergeHostingGoal = TabletHostingGoal.ALWAYS; + } else if (goals.contains(TabletHostingGoal.NEVER)) { + mergeHostingGoal = TabletHostingGoal.NEVER; + } + return mergeHostingGoal; + } + + // Divide each new DFV in half and make sure the sum equals the original + @VisibleForTesting + static Pair computeNewDfv(DataFileValue value) { + final DataFileValue file1Value = new DataFileValue(Math.max(1, value.getSize() / 2), + Math.max(1, value.getNumEntries() / 2), value.getTime()); + + final DataFileValue file2Value = + new DataFileValue(Math.max(1, value.getSize() - file1Value.getSize()), + Math.max(1, value.getNumEntries() - file1Value.getNumEntries()), value.getTime()); + + return new Pair<>(file1Value, file2Value); + } + + private Optional loadTabletMetadata(Manager manager, TableId tabletId, + final Text row, TabletMetadata.ColumnType... columns) { + try (TabletsMetadata tabletsMetadata = manager.getContext().getAmple().readTablets() + .forTable(tabletId).overlapping(row, true, row).fetch(columns).build()) { + return tabletsMetadata.stream().findFirst(); + } + } + + // This method is used to detect if a tablet needs to be split/chopped for a delete + // Instead of performing a split or chop compaction, the tablet will have its files fenced. + private boolean needsFencingForDeletion(MergeInfo info, KeyExtent keyExtent) { + // Does this extent cover the end points of the delete? + final Predicate isWithin = r -> r != null && keyExtent.contains(r); + final Predicate isNotBoundary = + r -> !r.equals(keyExtent.endRow()) && !r.equals(keyExtent.prevEndRow()); + final KeyExtent deleteRange = info.getExtent(); + + return (keyExtent.overlaps(deleteRange) && Stream + .of(deleteRange.prevEndRow(), deleteRange.endRow()).anyMatch(isWithin.and(isNotBoundary))) + || info.needsToBeChopped(keyExtent); + } + + // Instead of splitting or chopping tablets for a delete we instead create ranges + // to exclude the portion of the tablet that should be deleted + private Text followingRow(Text row) { + if (row == null) { + return null; + } + return new Key(row).followingKey(PartialKey.ROW).getRow(); + } + + // Instead of splitting or chopping tablets for a delete we instead create ranges + // to exclude the portion of the tablet that should be deleted + private List createRangesForDeletion(TabletMetadata tabletMetadata, + final KeyExtent deleteRange) { + final KeyExtent tabletExtent = tabletMetadata.getExtent(); + + // If the delete range wholly contains the tablet being deleted then there is no range to clip + // files to because the files should be completely dropped. + Preconditions.checkArgument(!deleteRange.contains(tabletExtent), "delete range:%s tablet:%s", + deleteRange, tabletExtent); + + final List ranges = new ArrayList<>(); + + if (deleteRange.overlaps(tabletExtent)) { + if (deleteRange.prevEndRow() != null + && tabletExtent.contains(followingRow(deleteRange.prevEndRow()))) { + log.trace("Fencing tablet {} files to ({},{}]", tabletExtent, tabletExtent.prevEndRow(), + deleteRange.prevEndRow()); + ranges.add(new Range(tabletExtent.prevEndRow(), false, deleteRange.prevEndRow(), true)); + } + + // This covers the case of when a deletion range overlaps the last tablet. We need to create a + // range that excludes the deletion. + if (deleteRange.endRow() != null + && tabletMetadata.getExtent().contains(deleteRange.endRow())) { + log.trace("Fencing tablet {} files to ({},{}]", tabletExtent, deleteRange.endRow(), + tabletExtent.endRow()); + ranges.add(new Range(deleteRange.endRow(), false, tabletExtent.endRow(), true)); + } + } else { + log.trace("Fencing tablet {} files to itself because it does not overlap delete range", + tabletExtent); + ranges.add(tabletExtent.toDataRange()); + } + + return ranges; + } + + private static boolean isFirstTabletInTable(KeyExtent tablet) { + return tablet != null && tablet.prevEndRow() == null; + } + + private static boolean isLastTabletInTable(KeyExtent tablet) { + return tablet != null && tablet.endRow() == null; + } + + private static boolean areContiguousTablets(KeyExtent firstTablet, KeyExtent lastTablet) { + return firstTablet != null && lastTablet != null + && Objects.equals(firstTablet.endRow(), lastTablet.prevEndRow()); + } + + private boolean hasTabletsToDelete(final KeyExtent firstTabletInRange, + final KeyExtent lastTableInRange) { + // If the tablets are equal (and not null) then the deletion range is just part of 1 tablet + // which will be fenced so there are no tablets to delete. The null check is because if both + // are null then we are just deleting everything, so we do have tablets to delete + if (Objects.equals(firstTabletInRange, lastTableInRange) && firstTabletInRange != null) { + log.trace( + "No tablets to delete, firstTablet {} equals lastTablet {} in deletion range and was fenced.", + firstTabletInRange, lastTableInRange); + return false; + // If the lastTablet of the deletion range is the first tablet of the table it has been fenced + // already so nothing to actually delete before it + } else if (isFirstTabletInTable(lastTableInRange)) { + log.trace( + "No tablets to delete, lastTablet {} in deletion range is the first tablet of the table and was fenced.", + lastTableInRange); + return false; + // If the firstTablet of the deletion range is the last tablet of the table it has been fenced + // already so nothing to actually delete after it + } else if (isLastTabletInTable(firstTabletInRange)) { + log.trace( + "No tablets to delete, firstTablet {} in deletion range is the last tablet of the table and was fenced.", + firstTabletInRange); + return false; + // If the firstTablet and lastTablet are contiguous tablets then there is nothing to delete as + // each will be fenced and nothing between + } else if (areContiguousTablets(firstTabletInRange, lastTableInRange)) { + log.trace( + "No tablets to delete, firstTablet {} and lastTablet {} in deletion range are contiguous and were fenced.", + firstTabletInRange, lastTableInRange); + return false; + } + + return true; + } + + static void deleteTablets(MergeInfo info, Range scanRange, BatchWriter bw, AccumuloClient client) + throws TableNotFoundException, MutationsRejectedException { + Scanner scanner; + Mutation m; + // Delete everything in the other tablets + // group all deletes into tablet into one mutation, this makes tablets + // either disappear entirely or not all.. this is important for the case + // where the process terminates in the loop below... + scanner = client.createScanner(info.getExtent().isMeta() ? RootTable.NAME : MetadataTable.NAME, + Authorizations.EMPTY); + log.debug("Deleting range {}", scanRange); + scanner.setRange(scanRange); + RowIterator rowIter = new RowIterator(scanner); + while (rowIter.hasNext()) { + Iterator> row = rowIter.next(); + m = null; + while (row.hasNext()) { + Map.Entry entry = row.next(); + Key key = entry.getKey(); + + if (m == null) { + m = new Mutation(key.getRow()); + } + + m.putDelete(key.getColumnFamily(), key.getColumnQualifier()); + log.debug("deleting entry {}", key); + } + bw.addMutation(m); + } + + bw.flush(); + } + + static KeyExtent getHighTablet(Manager manager, KeyExtent range) throws AccumuloException { + try { + AccumuloClient client = manager.getContext(); + Scanner scanner = client.createScanner(range.isMeta() ? RootTable.NAME : MetadataTable.NAME, + Authorizations.EMPTY); + MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner); + KeyExtent start = new KeyExtent(range.tableId(), range.endRow(), null); + scanner.setRange(new Range(start.toMetaRow(), null)); + Iterator> iterator = scanner.iterator(); + if (!iterator.hasNext()) { + throw new AccumuloException("No last tablet for a merge " + range); + } + Map.Entry entry = iterator.next(); + KeyExtent highTablet = KeyExtent.fromMetaPrevRow(entry); + if (!highTablet.tableId().equals(range.tableId())) { + throw new AccumuloException("No last tablet for merge " + range + " " + highTablet); + } + return highTablet; + } catch (Exception ex) { + throw new AccumuloException("Unexpected failure finding the last tablet for a merge " + range, + ex); + } + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/FinishTableRangeOp.java similarity index 86% rename from server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java rename to server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/FinishTableRangeOp.java index 9b73d4c538d..574335dcf3b 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOpWait.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/FinishTableRangeOp.java @@ -25,11 +25,12 @@ import org.apache.accumulo.manager.tableOps.ManagerRepo; import org.apache.accumulo.manager.tableOps.Utils; import org.apache.accumulo.server.manager.state.MergeInfo; -import org.apache.accumulo.server.manager.state.MergeState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** + * ELASTICITY_TODO edit these docs which are pre elasticity changes. Best done after #3763 + * * Merge makes things hard. * * Typically, a client will read the list of tablets, and begin an operation on that tablet at the @@ -46,26 +47,18 @@ * Normal operations, like bulk imports, will grab the read lock and prevent merges (writes) while * they run. Merge operations will lock out some operations while they run. */ -class TableRangeOpWait extends ManagerRepo { - private static final Logger log = LoggerFactory.getLogger(TableRangeOpWait.class); +class FinishTableRangeOp extends ManagerRepo { + private static final Logger log = LoggerFactory.getLogger(FinishTableRangeOp.class); private static final long serialVersionUID = 1L; private TableId tableId; private NamespaceId namespaceId; - public TableRangeOpWait(NamespaceId namespaceId, TableId tableId) { + public FinishTableRangeOp(NamespaceId namespaceId, TableId tableId) { this.tableId = tableId; this.namespaceId = namespaceId; } - @Override - public long isReady(long tid, Manager env) { - if (!env.getMergeInfo(tableId).getState().equals(MergeState.NONE)) { - return 50; - } - return 0; - } - @Override public Repo call(long tid, Manager manager) throws Exception { MergeInfo mergeInfo = manager.getMergeInfo(tableId); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeTablets.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeTablets.java new file mode 100644 index 00000000000..d6a7de9716f --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/MergeTablets.java @@ -0,0 +1,282 @@ +/* + * 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.manager.tableOps.merge; + +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import org.apache.accumulo.core.client.AccumuloClient; +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.BatchWriter; +import org.apache.accumulo.core.client.Scanner; +import org.apache.accumulo.core.client.admin.TabletHostingGoal; +import org.apache.accumulo.core.clientImpl.TabletHostingGoalUtil; +import org.apache.accumulo.core.data.Key; +import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.Range; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.dataImpl.KeyExtent; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.metadata.MetadataTable; +import org.apache.accumulo.core.metadata.RootTable; +import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.MetadataSchema; +import org.apache.accumulo.core.metadata.schema.MetadataTime; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.apache.accumulo.server.gc.AllVolumesDirectory; +import org.apache.accumulo.server.manager.state.MergeInfo; +import org.apache.accumulo.server.manager.state.MergeState; +import org.apache.accumulo.server.tablets.TabletTime; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Preconditions; + +public class MergeTablets extends ManagerRepo { + + private static final long serialVersionUID = 1L; + + private static final Logger log = LoggerFactory.getLogger(MergeTablets.class); + + private final NamespaceId namespaceId; + private final TableId tableId; + + public MergeTablets(NamespaceId namespaceId, TableId tableId) { + this.namespaceId = namespaceId; + this.tableId = tableId; + } + + @Override + public Repo call(long tid, Manager manager) throws Exception { + MergeInfo mergeInfo = manager.getMergeInfo(tableId); + Preconditions.checkState(mergeInfo.getState() == MergeState.MERGING); + Preconditions.checkState(!mergeInfo.isDelete()); + + var extent = mergeInfo.getExtent(); + long tabletCount; + + try (var tabletMeta = manager.getContext().getAmple().readTablets().forTable(extent.tableId()) + .overlapping(extent.prevEndRow(), extent.endRow()).fetch(TabletMetadata.ColumnType.PREV_ROW) + .checkConsistency().build()) { + tabletCount = tabletMeta.stream().count(); + } + + if (tabletCount > 1) { + mergeMetadataRecords(manager, mergeInfo); + } + + return new FinishTableRangeOp(namespaceId, tableId); + } + + private void mergeMetadataRecords(Manager manager, MergeInfo info) throws AccumuloException { + KeyExtent range = info.getExtent(); + log.debug("Merging metadata for {}", range); + KeyExtent stop = DeleteRows.getHighTablet(manager, range); + log.debug("Highest tablet is {}", stop); + Value firstPrevRowValue = null; + Text stopRow = stop.toMetaRow(); + Text start = range.prevEndRow(); + if (start == null) { + start = new Text(); + } + Range scanRange = new Range(MetadataSchema.TabletsSection.encodeRow(range.tableId(), start), + false, stopRow, false); + String targetSystemTable = MetadataTable.NAME; + if (range.isMeta()) { + targetSystemTable = RootTable.NAME; + } + Set goals = new HashSet<>(); + + AccumuloClient client = manager.getContext(); + + KeyExtent stopExtent = KeyExtent.fromMetaRow(stop.toMetaRow()); + KeyExtent previousKeyExtent = null; + KeyExtent lastExtent = null; + + try (BatchWriter bw = client.createBatchWriter(targetSystemTable)) { + long fileCount = 0; + // Make file entries in highest tablet + Scanner scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); + // Update to set the range to include the highest tablet + scanner.setRange(new Range(MetadataSchema.TabletsSection.encodeRow(range.tableId(), start), + false, stopRow, true)); + MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.fetch(scanner); + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME); + Mutation m = new Mutation(stopRow); + MetadataTime maxLogicalTime = null; + for (Map.Entry entry : scanner) { + Key key = entry.getKey(); + Value value = entry.getValue(); + + final KeyExtent keyExtent = KeyExtent.fromMetaRow(key.getRow()); + + // Keep track of the last Key Extent seen so we can use it to fence + // of RFiles when merging the metadata + if (lastExtent != null && !keyExtent.equals(lastExtent)) { + previousKeyExtent = lastExtent; + } + + // Special case to handle the highest/stop tablet, which is where files are + // merged to. The existing merge code won't delete files from this tablet + // so we need to handle the deletes in this tablet when fencing files. + // We may be able to make this simpler in the future. + if (keyExtent.equals(stopExtent)) { + if (previousKeyExtent != null && key.getColumnFamily() + .equals(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME)) { + + // Fence off existing files by the end row of the previous tablet and current tablet + final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier()); + // The end row should be inclusive for the current tablet and the previous end row + // should be exclusive for the start row + Range fenced = new Range(previousKeyExtent.endRow(), false, keyExtent.endRow(), true); + + // Clip range if exists + fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced; + + final StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); + // If the existing metadata does not match then we need to delete the old + // and replace with a new range + if (!existing.equals(newFile)) { + m.putDelete(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME, + existing.getMetadataText()); + m.put(key.getColumnFamily(), newFile.getMetadataText(), value); + } + + fileCount++; + } + // For the highest tablet we only care about the DataFileColumnFamily + continue; + } + + // Handle metadata updates for all other tablets except the highest tablet + // Ranges are created for the files and then added to the highest tablet in + // the merge range. Deletes are handled later for the old files when the tablets + // are removed. + if (key.getColumnFamily().equals(MetadataSchema.TabletsSection.DataFileColumnFamily.NAME)) { + final StoredTabletFile existing = StoredTabletFile.of(key.getColumnQualifier()); + + // Fence off files by the previous tablet and current tablet that is being merged + // The end row should be inclusive for the current tablet and the previous end row should + // be exclusive for the start row. + Range fenced = new Range(previousKeyExtent != null ? previousKeyExtent.endRow() : null, + false, keyExtent.endRow(), true); + + // Clip range with the tablet range if the range already exists + fenced = existing.hasRange() ? existing.getRange().clip(fenced) : fenced; + + // Move the file and range to the last tablet + StoredTabletFile newFile = StoredTabletFile.of(existing.getPath(), fenced); + m.put(key.getColumnFamily(), newFile.getMetadataText(), value); + + fileCount++; + } else if (MetadataSchema.TabletsSection.TabletColumnFamily.PREV_ROW_COLUMN.hasColumns(key) + && firstPrevRowValue == null) { + log.debug("prevRow entry for lowest tablet is {}", value); + firstPrevRowValue = new Value(value); + } else if (MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.hasColumns(key)) { + maxLogicalTime = + TabletTime.maxMetadataTime(maxLogicalTime, MetadataTime.parse(value.toString())); + } else if (MetadataSchema.TabletsSection.ServerColumnFamily.DIRECTORY_COLUMN + .hasColumns(key)) { + var allVolumesDir = new AllVolumesDirectory(range.tableId(), value.toString()); + bw.addMutation(manager.getContext().getAmple().createDeleteMutation(allVolumesDir)); + } else if (MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.hasColumns(key)) { + TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(value); + goals.add(thisGoal); + } + + lastExtent = keyExtent; + } + + // read the logical time from the last tablet in the merge range, it is not included in + // the loop above + scanner = client.createScanner(targetSystemTable, Authorizations.EMPTY); + scanner.setRange(new Range(stopRow)); + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.fetch(scanner); + MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.fetch(scanner); + scanner.fetchColumnFamily(MetadataSchema.TabletsSection.ExternalCompactionColumnFamily.NAME); + Set extCompIds = new HashSet<>(); + for (Map.Entry entry : scanner) { + if (MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN + .hasColumns(entry.getKey())) { + maxLogicalTime = TabletTime.maxMetadataTime(maxLogicalTime, + MetadataTime.parse(entry.getValue().toString())); + } else if (MetadataSchema.TabletsSection.ExternalCompactionColumnFamily.NAME + .equals(entry.getKey().getColumnFamily())) { + extCompIds.add(entry.getKey().getColumnQualifierData().toString()); + } else if (MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN + .hasColumns(entry.getKey())) { + TabletHostingGoal thisGoal = TabletHostingGoalUtil.fromValue(entry.getValue()); + goals.add(thisGoal); + } + } + + if (maxLogicalTime != null) { + MetadataSchema.TabletsSection.ServerColumnFamily.TIME_COLUMN.put(m, + new Value(maxLogicalTime.encode())); + } + + // delete any entries for external compactions + extCompIds.forEach(ecid -> m + .putDelete(MetadataSchema.TabletsSection.ExternalCompactionColumnFamily.STR_NAME, ecid)); + + // Set the TabletHostingGoal for this tablet based on the goals of the other tablets in + // the merge range. Always takes priority over never. + TabletHostingGoal mergeHostingGoal = DeleteRows.getMergeHostingGoal(range, goals); + MetadataSchema.TabletsSection.HostingColumnFamily.GOAL_COLUMN.put(m, + TabletHostingGoalUtil.toValue(mergeHostingGoal)); + + if (!m.getUpdates().isEmpty()) { + bw.addMutation(m); + } + + bw.flush(); + + log.debug("Moved {} files to {}", fileCount, stop); + + if (firstPrevRowValue == null) { + log.debug("tablet already merged"); + return; + } + + stop = new KeyExtent(stop.tableId(), stop.endRow(), + MetadataSchema.TabletsSection.TabletColumnFamily.decodePrevEndRow(firstPrevRowValue)); + Mutation updatePrevRow = + MetadataSchema.TabletsSection.TabletColumnFamily.createPrevRowMutation(stop); + log.debug("Setting the prevRow for last tablet: {}", stop); + bw.addMutation(updatePrevRow); + bw.flush(); + + DeleteRows.deleteTablets(info, scanRange, bw, client); + + } catch (Exception ex) { + throw new AccumuloException(ex); + } + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java index 2990b9e59f7..fa1f587e3a9 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/TableRangeOp.java @@ -86,12 +86,14 @@ public Repo call(long tid, Manager env) throws Exception { MergeInfo info = env.getMergeInfo(tableId); + // ELASTICITY_TODO can remove MergeState and MergeInfo once opid is set, these only exists now + // to get tablets unassigned. Once an opid is set on a tablet it will be unassigned. See #3763 if (info.getState() == MergeState.NONE) { KeyExtent range = new KeyExtent(tableId, end, start); - env.setMergeState(new MergeInfo(range, op), MergeState.STARTED); + env.setMergeState(new MergeInfo(range, op), MergeState.WAITING_FOR_OFFLINE); } - return new TableRangeOpWait(namespaceId, tableId); + return new WaitForOffline(namespaceId, tableId); } @Override diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/WaitForOffline.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/WaitForOffline.java new file mode 100644 index 00000000000..c55ff213c5c --- /dev/null +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/merge/WaitForOffline.java @@ -0,0 +1,81 @@ +/* + * 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.manager.tableOps.merge; + +import org.apache.accumulo.core.data.NamespaceId; +import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.fate.FateTxId; +import org.apache.accumulo.core.fate.Repo; +import org.apache.accumulo.core.metadata.schema.TabletMetadata; +import org.apache.accumulo.manager.Manager; +import org.apache.accumulo.manager.tableOps.ManagerRepo; +import org.apache.accumulo.server.manager.state.MergeInfo; +import org.apache.accumulo.server.manager.state.MergeState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class WaitForOffline extends ManagerRepo { + + private static final Logger log = LoggerFactory.getLogger(WaitForOffline.class); + + private static final long serialVersionUID = 1L; + + private final NamespaceId namespaceId; + private final TableId tableId; + + public WaitForOffline(NamespaceId namespaceId, TableId tableId) { + this.namespaceId = namespaceId; + this.tableId = tableId; + } + + @Override + public long isReady(long tid, Manager env) throws Exception { + MergeInfo mergeInfo = env.getMergeInfo(tableId); + var extent = mergeInfo.getExtent(); + + long tabletsWithLocations; + + try (var tabletMeta = env.getContext().getAmple().readTablets().forTable(extent.tableId()) + .overlapping(extent.prevEndRow(), extent.endRow()) + .fetch(TabletMetadata.ColumnType.PREV_ROW, TabletMetadata.ColumnType.LOCATION) + .checkConsistency().build()) { + tabletsWithLocations = tabletMeta.stream().filter(tm -> tm.getLocation() != null).count(); + } + + log.info("{} waiting for {} tablets with locations", FateTxId.formatTid(tid), + tabletsWithLocations); + + if (tabletsWithLocations > 0) { + return 1000; + } else { + return 0; + } + } + + @Override + public Repo call(long tid, Manager env) throws Exception { + MergeInfo mergeInfo = env.getMergeInfo(tableId); + env.setMergeState(mergeInfo, MergeState.MERGING); + if (mergeInfo.isDelete()) { + return new DeleteRows(namespaceId, tableId); + } else { + return new MergeTablets(namespaceId, tableId); + } + } +} diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java index 5e6505f58db..6704ef6d545 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/PreSplit.java @@ -98,7 +98,7 @@ public long isReady(long tid, Manager manager) throws Exception { try (var tabletsMutator = manager.getContext().getAmple().conditionallyMutateTablets()) { tabletsMutator.mutateTablet(splitInfo.getOriginal()).requireAbsentOperation() - .requireSame(tabletMetadata, LOCATION, PREV_ROW).putOperation(opid) + .requireSame(tabletMetadata, LOCATION).putOperation(opid) .submit(tmeta -> opid.equals(tmeta.getOperationId())); Map results = tabletsMutator.process(); diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/SplitInfo.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/SplitInfo.java index 52051a0dc0d..b8f8c7adff5 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/SplitInfo.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/SplitInfo.java @@ -73,8 +73,6 @@ SortedSet getTablets() { TreeSet tablets = new TreeSet<>(); - double sum = 0; - for (var split : getSplits()) { var extent = new KeyExtent(getOriginal().tableId(), split, prev); prev = split; diff --git a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/UpdateTablets.java b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/UpdateTablets.java index 28131f795e3..246c73d8cee 100644 --- a/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/UpdateTablets.java +++ b/server/manager/src/main/java/org/apache/accumulo/manager/tableOps/split/UpdateTablets.java @@ -217,7 +217,7 @@ private void updateExistingTablet(long tid, Manager manager, TabletMetadata tabl var newExtent = newTablets.last(); var mutator = tabletsMutator.mutateTablet(splitInfo.getOriginal()).requireOperation(opid) - .requirePrevEndRow(splitInfo.getOriginal().prevEndRow()).requireAbsentLocation(); + .requireAbsentLocation(); mutator.putPrevEndRow(newExtent.prevEndRow()); diff --git a/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/DeleteRowsTest.java similarity index 87% rename from server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java rename to server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/DeleteRowsTest.java index 35026efbe96..0dfe5ee6a58 100644 --- a/server/manager/src/test/java/org/apache/accumulo/manager/TabletGroupWatcherTest.java +++ b/server/manager/src/test/java/org/apache/accumulo/manager/tableOps/merge/DeleteRowsTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.accumulo.manager; +package org.apache.accumulo.manager.tableOps.merge; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -24,12 +24,12 @@ import org.apache.accumulo.core.util.Pair; import org.junit.jupiter.api.Test; -public class TabletGroupWatcherTest { +public class DeleteRowsTest { @Test public void testComputeNewDfvEven() { DataFileValue original = new DataFileValue(20, 10, 100); - Pair newValues = TabletGroupWatcher.computeNewDfv(original); + Pair newValues = DeleteRows.computeNewDfv(original); assertEquals(10, newValues.getFirst().getSize()); assertEquals(5, newValues.getFirst().getNumEntries()); @@ -42,7 +42,7 @@ public void testComputeNewDfvEven() { @Test public void testComputeNewDfvOdd() { DataFileValue original = new DataFileValue(21, 11, 100); - Pair newValues = TabletGroupWatcher.computeNewDfv(original); + Pair newValues = DeleteRows.computeNewDfv(original); assertEquals(10, newValues.getFirst().getSize()); assertEquals(5, newValues.getFirst().getNumEntries()); @@ -55,7 +55,7 @@ public void testComputeNewDfvOdd() { @Test public void testComputeNewDfvSmall() { DataFileValue original = new DataFileValue(1, 2, 100); - Pair newValues = TabletGroupWatcher.computeNewDfv(original); + Pair newValues = DeleteRows.computeNewDfv(original); assertEquals(1, newValues.getFirst().getSize()); assertEquals(1, newValues.getFirst().getNumEntries()); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java index f4cac95747e..0b4b2f21ffb 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/AssignmentHandler.java @@ -27,7 +27,6 @@ import java.util.concurrent.TimeUnit; import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.admin.TabletHostingGoal; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.manager.thrift.TabletLoadState; @@ -44,6 +43,7 @@ import org.apache.accumulo.tserver.TabletServerResourceManager.TabletResourceManager; import org.apache.accumulo.tserver.managermessage.TabletStatusMessage; import org.apache.accumulo.tserver.tablet.Tablet; +import org.apache.accumulo.tserver.tablet.Tablet.RefreshPurpose; import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -178,10 +178,14 @@ public void run() { && !tablet.minorCompactNow(MinorCompactionReason.RECOVERY)) { throw new RuntimeException("Minor compaction after recovery fails for " + extent); } + Assignment assignment = new Assignment(extent, server.getTabletSession(), tabletMetadata.getLast()); TabletStateStore.setLocation(server.getContext(), assignment); + // refresh the tablet metadata after setting the location (See #3358) + tablet.refreshMetadata(RefreshPurpose.LOAD); + synchronized (server.openingTablets) { synchronized (server.onlineTablets) { server.openingTablets.remove(extent); @@ -190,6 +194,7 @@ public void run() { server.recentlyUnloadedCache.remove(tablet.getExtent()); } } + tablet = null; // release this reference successful = true; } catch (Exception e) { @@ -208,9 +213,6 @@ public void run() { if (successful) { server.enqueueManagerMessage(new TabletStatusMessage(TabletLoadState.LOADED, extent)); - if (tabletMetadata.getHostingGoal() == TabletHostingGoal.ONDEMAND) { - server.insertOnDemandAccessTime(extent); - } } else { synchronized (server.unopenedTablets) { synchronized (server.openingTablets) { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 0555aa72cf2..8a68b2a7cc7 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -156,8 +156,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.net.HostAndPort; -import io.opentelemetry.context.Scope; - public class TabletServer extends AbstractServer implements TabletHostingServer { private static final Logger log = LoggerFactory.getLogger(TabletServer.class); @@ -195,8 +193,6 @@ public PausedCompactionMetrics getPausedCompactionMetrics() { private final AtomicLong syncCounter = new AtomicLong(0); final OnlineTablets onlineTablets = new OnlineTablets(); - private final Map onDemandTabletAccessTimes = - Collections.synchronizedMap(new HashMap<>()); final SortedSet unopenedTablets = Collections.synchronizedSortedSet(new TreeSet<>()); final SortedSet openingTablets = Collections.synchronizedSortedSet(new TreeSet<>()); final Map recentlyUnloadedCache = Collections.synchronizedMap(new LRUMap<>(1000)); @@ -1000,8 +996,8 @@ public SortedMap getOnlineTablets() { @Override public Tablet getOnlineTablet(KeyExtent extent) { Tablet t = onlineTablets.snapshot().get(extent); - if (t != null && t.isOnDemand()) { - updateOnDemandAccessTime(extent); + if (t != null) { + t.setLastAccessTime(); } return t; } @@ -1142,28 +1138,6 @@ public int getOnDemandOnlineUnloadedForLowMemory() { return onDemandUnloadedLowMemory.get(); } - // called from AssignmentHandler - public void insertOnDemandAccessTime(KeyExtent extent) { - if (extent.isMeta()) { - return; - } - onDemandTabletAccessTimes.putIfAbsent(extent, new AtomicLong(System.nanoTime())); - } - - // called from getOnlineExtent - private void updateOnDemandAccessTime(KeyExtent extent) { - final long currentTime = System.nanoTime(); - AtomicLong l = onDemandTabletAccessTimes.get(extent); - if (l != null) { - l.set(currentTime); - } - } - - // called from UnloadTabletHandler - public void removeOnDemandAccessTime(KeyExtent extent) { - onDemandTabletAccessTimes.remove(extent); - } - private boolean isTabletInUse(KeyExtent extent) { // Don't call getOnlineTablet as that will update the last access time final Tablet t = onlineTablets.snapshot().get(extent); @@ -1182,42 +1156,29 @@ public void evaluateOnDemandTabletsForUnload() { final SortedMap online = getOnlineTablets(); - // Find and remove access time entries for KeyExtents - // that are no longer in the onlineTablets collection - Set missing = onDemandTabletAccessTimes.keySet().stream() - .filter(k -> !online.containsKey(k)).collect(Collectors.toSet()); - if (!missing.isEmpty()) { - log.debug("Removing onDemandAccessTimes for tablets as tablets no longer online: {}", - missing); - missing.forEach(onDemandTabletAccessTimes::remove); - if (onDemandTabletAccessTimes.isEmpty()) { - return; - } - } - - // It's possible, from a tablet split or merge for example, - // that there is an on-demand tablet that is hosted for which - // we have no access time. Add any missing online on-demand - // tablets - online.forEach((k, v) -> { - if (v.isOnDemand() && !onDemandTabletAccessTimes.containsKey(k)) { - insertOnDemandAccessTime(k); + // Sort the extents so that we can process them by table. + final SortedMap sortedOnDemandExtents = new TreeMap<>(); + // We only want to operate on OnDemand Tablets + online.entrySet().forEach((e) -> { + if (e.getValue().isOnDemand()) { + sortedOnDemandExtents.put(e.getKey(), e.getValue().getLastAccessTime()); } }); - log.debug("Evaluating online on-demand tablets: {}", onDemandTabletAccessTimes); - - if (onDemandTabletAccessTimes.isEmpty()) { + if (sortedOnDemandExtents.isEmpty()) { return; } + log.debug("Evaluating online on-demand tablets: {}", sortedOnDemandExtents); + // If the TabletServer is running low on memory, don't call the SPI // plugin to evaluate which on-demand tablets to unload, just get the // on-demand tablet with the oldest access time and unload it. if (getContext().getLowMemoryDetector().isRunningLowOnMemory()) { final SortedMap timeSortedOnDemandExtents = new TreeMap<>(); - onDemandTabletAccessTimes.forEach((k, v) -> timeSortedOnDemandExtents.put(v.get(), k)); - Long oldestAccessTime = timeSortedOnDemandExtents.firstKey(); + long currTime = System.nanoTime(); + sortedOnDemandExtents.forEach((k, v) -> timeSortedOnDemandExtents.put(v - currTime, k)); + Long oldestAccessTime = timeSortedOnDemandExtents.lastKey(); KeyExtent oldestKeyExtent = timeSortedOnDemandExtents.get(oldestAccessTime); log.warn("Unloading on-demand tablet: {} for table: {} due to low memory", oldestKeyExtent, oldestKeyExtent.tableId()); @@ -1226,12 +1187,6 @@ public void evaluateOnDemandTabletsForUnload() { return; } - // onDemandTabletAccessTimes is a HashMap. Sort the extents - // so that we can process them by table. - final SortedMap sortedOnDemandExtents = - new TreeMap(); - sortedOnDemandExtents.putAll(onDemandTabletAccessTimes); - // The access times are updated when getOnlineTablet is called by other methods, // but may not necessarily capture whether or not the Tablet is currently being used. // For example, getOnlineTablet is called from startScan but not from continueScan. @@ -1276,7 +1231,7 @@ public void evaluateOnDemandTabletsForUnload() { }); tableIds.forEach(tid -> { Map subset = sortedOnDemandExtents.entrySet().stream() + Long> subset = sortedOnDemandExtents.entrySet().stream() .filter((e) -> e.getKey().tableId().equals(tid)) .collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue())); Set onDemandTabletsToUnload = new HashSet<>(); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java index 4a6081cd07f..fc7bbcca39f 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/UnloadTabletHandler.java @@ -107,9 +107,6 @@ public void run() { // exceptions server.recentlyUnloadedCache.put(extent, System.currentTimeMillis()); server.onlineTablets.remove(extent); - if (t.isOnDemand()) { - server.removeOnDemandAccessTime(extent); - } try { TServerInstance instance = server.getTabletSession(); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinCEnv.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinCEnv.java index ea683b04bf0..aaa80625501 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinCEnv.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/MinCEnv.java @@ -25,7 +25,6 @@ import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.tabletserver.thrift.TCompactionReason; -import org.apache.accumulo.core.util.ratelimit.RateLimiter; import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.compaction.FileCompactor; import org.apache.accumulo.server.iterators.SystemIteratorEnvironment; @@ -51,16 +50,6 @@ public IteratorUtil.IteratorScope getIteratorScope() { return IteratorUtil.IteratorScope.minc; } - @Override - public RateLimiter getReadLimiter() { - return null; - } - - @Override - public RateLimiter getWriteLimiter() { - return null; - } - @Override public SystemIteratorEnvironment createIteratorEnv(ServerContext context, AccumuloConfiguration acuTableConf, TableId tableId) { diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java index 789649758c3..f08922fa8c8 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/tablet/Tablet.java @@ -37,7 +37,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -63,6 +62,7 @@ import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.ReferencedTabletFile; import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.Ample; import org.apache.accumulo.core.metadata.schema.DataFileValue; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; @@ -104,6 +104,8 @@ import org.slf4j.LoggerFactory; import com.google.common.base.Preconditions; +import com.google.common.collect.Sets; +import com.google.common.collect.Sets.SetView; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import io.opentelemetry.api.trace.Span; @@ -178,6 +180,8 @@ enum CompactionState { private final int logId; + private volatile long lastAccessTime = System.nanoTime(); + public int getLogId() { return logId; } @@ -424,10 +428,6 @@ public void flush(long tableFlushID) { return; } - if (getMetadata().getFlushId().orElse(-1) >= tableFlushID) { - return; - } - if (isClosing() || isClosed() || isBeingDeleted() || getTabletMemory().memoryReservedForMinC()) { return; @@ -445,18 +445,37 @@ public void flush(long tableFlushID) { refreshLock.lock(); try { // if multiple threads were allowed to update this outside of a sync block, then it would - // be - // a race condition - // ELASTICITY_TODO use conditional mutations - MetadataTableUtil.updateTabletFlushID(extent, tableFlushID, context, - getTabletServer().getLock()); - // It is important the the refresh lock is held for the update above and the refresh below - // to avoid race conditions. - refreshMetadata(RefreshPurpose.FLUSH_ID_UPDATE); + // be a race condition + var lastTabletMetadata = getMetadata(); + + // Check flush id while holding refresh lock to prevent race condition with other threads + // in tablet reading and writing the tablets metadata. + if (lastTabletMetadata.getFlushId().orElse(-1) < tableFlushID) { + try (var tabletsMutator = getContext().getAmple().conditionallyMutateTablets()) { + var tablet = tabletsMutator.mutateTablet(extent) + .requireLocation(Location.current(tabletServer.getTabletSession())) + .requireSame(lastTabletMetadata, ColumnType.FLUSH_ID); + + tablet.putFlushId(tableFlushID); + tablet.putZooLock(context.getZooKeeperRoot(), getTabletServer().getLock()); + tablet + .submit(tabletMetadata -> tabletMetadata.getFlushId().orElse(-1) == tableFlushID); + + var result = tabletsMutator.process().get(extent); + + if (result.getStatus() != Ample.ConditionalResult.Status.ACCEPTED) { + throw new IllegalStateException("Failed to update flush id " + extent + " " + + tabletServer.getTabletSession() + " " + tableFlushID); + } + } + + // It is important the the refresh lock is held for the update above and the refresh + // below to avoid race conditions. + refreshMetadata(RefreshPurpose.FLUSH_ID_UPDATE); + } } finally { refreshLock.unlock(); } - } else if (initiateMinor) { initiateMinorCompaction(tableFlushID, MinorCompactionReason.USER); } @@ -1281,20 +1300,65 @@ public Optional updateTabletDataFile(long maxCommittedTime, ReferencedTabletFile newDatafile, DataFileValue dfv, Set unusedWalLogs, long flushId) { - // expect time to only move forward from what was recently seen in metadata table - Preconditions.checkArgument(maxCommittedTime >= getMetadata().getTime().getTime()); + Preconditions.checkState(refreshLock.isHeldByCurrentThread()); + + // Read these once in case of buggy race conditions will get consistent logging. If all other + // code is locking properly these should not change during this method. + var lastTabletMetadata = getMetadata(); + var expectedTime = lastTabletMetadata.getTime(); + + // Expect time to only move forward from what was recently seen in metadata table. + Preconditions.checkArgument(maxCommittedTime >= expectedTime.getTime()); + + // The tablet time is used to determine if the write succeeded, in order to do this the tablet + // time needs to be different from what is currently stored in the metadata table. + while (maxCommittedTime == expectedTime.getTime()) { + var nextTime = tabletTime.getAndUpdateTime(); + Preconditions.checkState(nextTime >= maxCommittedTime); + if (nextTime > maxCommittedTime) { + maxCommittedTime++; + } + } + + try (var tabletsMutator = getContext().getAmple().conditionallyMutateTablets()) { + var tablet = tabletsMutator.mutateTablet(extent) + .requireLocation(Location.current(tabletServer.getTabletSession())) + .requireSame(lastTabletMetadata, ColumnType.TIME); + + Optional newFile = Optional.empty(); + + // if entries are present, write to path to metadata table + if (dfv.getNumEntries() > 0) { + tablet.putFile(newDatafile, dfv); + newFile = Optional.of(newDatafile.insert()); - // ELASTICITY_TODO use conditional mutation, can check time and location + ManagerMetadataUtil.updateLastForCompactionMode(getContext(), tablet, lastLocation, + tabletServer.getTabletSession()); + } + + var newTime = tabletTime.getMetadataTime(maxCommittedTime); + tablet.putTime(newTime); + + tablet.putFlushId(flushId); + + unusedWalLogs.forEach(tablet::deleteWal); - // ELASTICITY_TODO minor compaction will need something like the bulk import loaded column - // to avoid : partial write, compact of file in partial write, and then another write of the - // file - // leading to the file being added twice. + tablet.putZooLock(getContext().getZooKeeperRoot(), tabletServer.getLock()); - return ManagerMetadataUtil.updateTabletDataFile(getTabletServer().getContext(), extent, - newDatafile, dfv, tabletTime.getMetadataTime(maxCommittedTime), - tabletServer.getTabletSession(), tabletServer.getLock(), unusedWalLogs, lastLocation, - flushId); + // When trying to determine if write was successful, check if the time was updated. Can not + // check if the new file exists because of two reasons. First, it could be compacted away + // between the write and check. Second, some flushes do not produce a file. + tablet.submit(tabletMetadata -> tabletMetadata.getTime().equals(newTime)); + + if (tabletsMutator.process().get(extent).getStatus() + != Ample.ConditionalResult.Status.ACCEPTED) { + // Include the things that could have caused the write to fail. + throw new IllegalStateException("Unable to write minor compaction. " + extent + " " + + tabletServer.getTabletSession() + " " + expectedTime); + } + + return newFile; + } } @Override @@ -1358,7 +1422,7 @@ public boolean isOnDemand() { // The purpose of this lock is to prevent race conditions between concurrent refresh RPC calls and // between minor compactions and refresh calls. - private final Lock refreshLock = new ReentrantLock(); + private final ReentrantLock refreshLock = new ReentrantLock(); void bringMinorCompactionOnline(ReferencedTabletFile tmpDatafile, ReferencedTabletFile newDatafile, DataFileValue dfv, CommitSession commitSession, @@ -1455,7 +1519,7 @@ void bringMinorCompactionOnline(ReferencedTabletFile tmpDatafile, } public enum RefreshPurpose { - MINC_COMPLETION, REFRESH_RPC, FLUSH_ID_UPDATE + MINC_COMPLETION, REFRESH_RPC, FLUSH_ID_UPDATE, LOAD } public void refreshMetadata(RefreshPurpose refreshPurpose) { @@ -1466,9 +1530,25 @@ public void refreshMetadata(RefreshPurpose refreshPurpose) { // scans TabletMetadata tabletMetadata = getContext().getAmple().readTablet(getExtent()); + Preconditions.checkState(tabletMetadata != null, "Tablet no longer exits %s", getExtent()); + Preconditions.checkState( + Location.current(tabletServer.getTabletSession()).equals(tabletMetadata.getLocation()), + "Tablet %s location %s is not this tserver %s", getExtent(), tabletMetadata.getLocation(), + tabletServer.getTabletSession()); + synchronized (this) { + var prevMetadata = latestMetadata; latestMetadata = tabletMetadata; + if (log.isDebugEnabled() && !prevMetadata.getFiles().equals(latestMetadata.getFiles())) { + SetView removed = + Sets.difference(prevMetadata.getFiles(), latestMetadata.getFiles()); + SetView added = + Sets.difference(latestMetadata.getFiles(), prevMetadata.getFiles()); + log.debug("Tablet {} was refreshed. Files removed: {} Files added: {}", this.getExtent(), + removed, added); + } + if (refreshPurpose == RefreshPurpose.MINC_COMPLETION) { // Atomically replace the in memory map with the new file. Before this synch block a scan // starting would see the in memory map. After this synch block it should see the file in @@ -1484,7 +1564,7 @@ public void refreshMetadata(RefreshPurpose refreshPurpose) { // important to call this after updating latestMetadata and tabletMemory computeNumEntries(); - } else if (!latestMetadata.getFilesMap().equals(tabletMetadata.getFilesMap())) { + } else if (!prevMetadata.getFilesMap().equals(latestMetadata.getFilesMap())) { // the files changed, incrementing this will cause scans to switch data sources dataSourceDeletions.incrementAndGet(); @@ -1501,4 +1581,13 @@ public void refreshMetadata(RefreshPurpose refreshPurpose) { scanfileManager.removeFilesAfterScan(getMetadata().getScans()); } } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public void setLastAccessTime() { + this.lastAccessTime = System.nanoTime(); + } + } diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/DefaultOnDemandTabletUnloaderTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/DefaultOnDemandTabletUnloaderTest.java index eeb27d75b81..3db9c0e3e86 100644 --- a/server/tserver/src/test/java/org/apache/accumulo/tserver/DefaultOnDemandTabletUnloaderTest.java +++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/DefaultOnDemandTabletUnloaderTest.java @@ -31,7 +31,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; import org.apache.accumulo.core.data.TableId; import org.apache.accumulo.core.dataImpl.KeyExtent; @@ -63,13 +62,13 @@ public void evaluationTest() { expect(tconf.get(DefaultOnDemandTabletUnloader.INACTIVITY_THRESHOLD)) .andReturn(inactivityTimeSeconds); expect(tconf.newDeriver(anyObject())).andReturn(Map::of).anyTimes(); - Map online = new HashMap<>(); + Map online = new HashMap<>(); // add an extent whose last access time is less than the currentTime - inactivityTime final KeyExtent activeExtent = new KeyExtent(tid, new Text("m"), new Text("a")); - online.put(activeExtent, new AtomicLong(currentTime - inactivityTime - 10)); + online.put(activeExtent, currentTime - inactivityTime - 10); // add an extent whose last access time is greater than the currentTime - inactivityTime final KeyExtent inactiveExtent = new KeyExtent(tid, new Text("z"), new Text("m")); - online.put(inactiveExtent, new AtomicLong(currentTime - inactivityTime + 10)); + online.put(inactiveExtent, currentTime - inactivityTime + 10); Set onDemandTabletsToUnload = new HashSet<>(); replay(context, tconf); diff --git a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java index cb88399c98b..2e6ee5054a7 100644 --- a/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java +++ b/test/src/main/java/org/apache/accumulo/test/AuditMessageIT.java @@ -61,6 +61,7 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -443,6 +444,7 @@ public void testDataOperationsAudits() throws AccumuloSecurityException, Accumul } @Test + @Disabled // ELASTICITY_TODO public void testDeniedAudits() throws AccumuloSecurityException, AccumuloException, TableExistsException, TableNotFoundException, IOException { diff --git a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java index d1b328178d3..dabd72c6bd5 100644 --- a/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java +++ b/test/src/main/java/org/apache/accumulo/test/CleanWalIT.java @@ -44,12 +44,14 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.collect.Iterators; +@Disabled // ELASTICITY_TODO public class CleanWalIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(CleanWalIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java b/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java index 1cd5ce06cd0..cec2e224c6d 100644 --- a/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ComprehensiveIT.java @@ -207,6 +207,12 @@ public void testFlushAndIterator() throws Exception { Wait.waitFor(() -> expected.equals(scan(client, table, AUTHORIZATIONS))); verifyData(client, table, AUTHORIZATIONS, expected); + + // flush a table with no unflushed data, tablet servers take a different code path for this + // case + client.tableOperations().flush(table, null, null, true); + + verifyData(client, table, AUTHORIZATIONS, expected); } } diff --git a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java index 45bd1a5f7ab..9c17b4acb04 100644 --- a/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java +++ b/test/src/main/java/org/apache/accumulo/test/GarbageCollectWALIT.java @@ -39,8 +39,10 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.fs.RemoteIterator; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class GarbageCollectWALIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java index bd986f519d5..71a670ed39d 100644 --- a/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java +++ b/test/src/main/java/org/apache/accumulo/test/LargeSplitRowIT.java @@ -51,6 +51,7 @@ import org.apache.accumulo.test.util.Wait; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import org.slf4j.Logger; @@ -74,6 +75,7 @@ public void configure(MiniAccumuloConfigImpl cfg, Configuration hadoopCoreSite) // User added split @Test + @Disabled // ELASTICITY_TODO public void userAddedSplit() throws Exception { log.info("User added split"); diff --git a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java index f1878cbfb2b..fa301cb5ae4 100644 --- a/test/src/main/java/org/apache/accumulo/test/LocatorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/LocatorIT.java @@ -52,8 +52,10 @@ import org.apache.accumulo.test.functional.ManagerAssignmentIT; import org.apache.accumulo.test.util.Wait; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class LocatorIT extends AccumuloClusterHarness { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java index 249c7072f90..c4094cfe08a 100644 --- a/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ManagerRepairsDualAssignmentIT.java @@ -51,8 +51,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class ManagerRepairsDualAssignmentIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java index 11e697e899e..8487f17877f 100644 --- a/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MetaRecoveryIT.java @@ -40,11 +40,13 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterators; // ACCUMULO-3211 +@Disabled // ELASTICITY_TODO public class MetaRecoveryIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java index 0fdc25c8c21..79613d9d140 100644 --- a/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/MultiTableRecoveryIT.java @@ -44,8 +44,10 @@ import org.apache.accumulo.test.functional.ConfigurableMacBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class MultiTableRecoveryIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java index 7657b0c2603..1573cf98702 100644 --- a/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/RecoveryCompactionsAreFlushesIT.java @@ -40,9 +40,11 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; // Accumulo3010 +@Disabled // ELASTICITY_TODO public class RecoveryCompactionsAreFlushesIT extends AccumuloClusterHarness { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/SampleIT.java b/test/src/main/java/org/apache/accumulo/test/SampleIT.java index 369444ac0a5..c4f4a5fbd6a 100644 --- a/test/src/main/java/org/apache/accumulo/test/SampleIT.java +++ b/test/src/main/java/org/apache/accumulo/test/SampleIT.java @@ -18,6 +18,7 @@ */ package org.apache.accumulo.test; +import static org.apache.accumulo.test.util.FileMetadataUtil.countFiles; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -26,11 +27,13 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; +import java.util.stream.Collectors; import org.apache.accumulo.core.client.Accumulo; import org.apache.accumulo.core.client.AccumuloClient; @@ -52,6 +55,7 @@ import org.apache.accumulo.core.client.sample.SamplerConfiguration; import org.apache.accumulo.core.clientImpl.ClientContext; import org.apache.accumulo.core.clientImpl.OfflineScanner; +import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Mutation; @@ -63,6 +67,8 @@ import org.apache.accumulo.core.iterators.WrappingIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.harness.AccumuloClusterHarness; +import org.apache.accumulo.test.util.FileMetadataUtil; +import org.apache.hadoop.io.Text; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterables; @@ -120,14 +126,75 @@ public void init(SortedKeyValueIterator source, Map op } } + @Test + public void testSampleFencing() throws Exception { + + try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { + String tableName = getUniqueNames(1)[0]; + String clone = tableName + "_clone"; + + createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1)); + + BatchWriter bw = client.createBatchWriter(tableName); + + TreeMap expected = new TreeMap<>(); + writeData(bw, SC1, expected); + assertEquals(20, expected.size()); + + Scanner scanner = client.createScanner(tableName, Authorizations.EMPTY); + Scanner isoScanner = + new IsolatedScanner(client.createScanner(tableName, Authorizations.EMPTY)); + Scanner csiScanner = + new ClientSideIteratorScanner(client.createScanner(tableName, Authorizations.EMPTY)); + scanner.setSamplerConfiguration(SC1); + csiScanner.setSamplerConfiguration(SC1); + isoScanner.setSamplerConfiguration(SC1); + isoScanner.setBatchSize(10); + + try (BatchScanner bScanner = client.createBatchScanner(tableName)) { + bScanner.setSamplerConfiguration(SC1); + bScanner.setRanges(Arrays.asList(new Range())); + + check(expected, scanner, bScanner, isoScanner, csiScanner); + + client.tableOperations().flush(tableName, null, null, true); + + // Fence off the data to a Range that is a subset of the original data + Range fenced = new Range(new Text(String.format("r_%06d", 3000)), + new Text(String.format("r_%06d", 6000))); + FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, Set.of(fenced)); + assertEquals(1, countFiles(getServerContext(), tableName)); + + // Build the map of expected values to be seen by filtering out keys not in the fenced range + TreeMap fenceExpected = + expected.entrySet().stream().filter(entry -> fenced.contains(entry.getKey())).collect( + Collectors.toMap(Entry::getKey, Entry::getValue, (v1, v2) -> v1, TreeMap::new)); + + Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1); + + // verify only the correct values in the fenced range are seen + check(fenceExpected, scanner, bScanner, isoScanner, csiScanner, oScanner); + } + } + } + @Test public void testBasic() throws Exception { + testBasic(Set.of()); + } + + @Test + public void testBasicWithFencedFiles() throws Exception { + testBasic(createRanges()); + } + + private void testBasic(Set fileRanges) throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; String clone = tableName + "_clone"; - client.tableOperations().create(tableName, new NewTableConfiguration().enableSampling(SC1)); + createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1)); BatchWriter bw = client.createBatchWriter(tableName); @@ -153,6 +220,12 @@ public void testBasic() throws Exception { client.tableOperations().flush(tableName, null, null, true); + // Split files into ranged files if provided + if (!fileRanges.isEmpty()) { + FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges); + assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName)); + } + Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1); check(expected, scanner, bScanner, isoScanner, csiScanner, oScanner); @@ -293,11 +366,20 @@ private void setRange(Range range, List scanners) { @Test public void testIterator() throws Exception { + testIterator(Set.of()); + } + + @Test + public void testIteratorFencedFiles() throws Exception { + testIterator(createRanges()); + } + + private void testIterator(Set fileRanges) throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; String clone = tableName + "_clone"; - client.tableOperations().create(tableName, new NewTableConfiguration().enableSampling(SC1)); + createTable(client, tableName, new NewTableConfiguration().enableSampling(SC1)); TreeMap expected = new TreeMap<>(); try (BatchWriter bw = client.createBatchWriter(tableName)) { @@ -346,6 +428,12 @@ public void testIterator() throws Exception { // flush an rerun same test against files client.tableOperations().flush(tableName, null, null, true); + // Split files into ranged files if provided + if (!fileRanges.isEmpty()) { + FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges); + assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName)); + } + oScanner = newOfflineScanner(client, tableName, clone, null); oScanner.addScanIterator(new IteratorSetting(100, IteratorThatUsesSample.class)); scanners = Arrays.asList(scanner, isoScanner, bScanner, csiScanner, oScanner); @@ -401,12 +489,21 @@ private void setSamplerConfig(SamplerConfiguration sc, ScannerBase... scanners) @Test public void testSampleNotPresent() throws Exception { + testSampleNotPresent(Set.of()); + } + + @Test + public void testSampleNotPresentFencedFiles() throws Exception { + testSampleNotPresent(createRanges()); + } + + private void testSampleNotPresent(Set fileRanges) throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; String clone = tableName + "_clone"; - client.tableOperations().create(tableName); + createTable(client, tableName, new NewTableConfiguration()); TreeMap expected = new TreeMap<>(); try (BatchWriter bw = client.createBatchWriter(tableName)) { @@ -425,12 +522,17 @@ public void testSampleNotPresent() throws Exception { client.tableOperations().flush(tableName, null, null, true); + // Split files into ranged files if provided + if (!fileRanges.isEmpty()) { + FileMetadataUtil.splitFilesIntoRanges(getServerContext(), tableName, fileRanges); + assertEquals(fileRanges.size(), countFiles(getServerContext(), tableName)); + } + Scanner oScanner = newOfflineScanner(client, tableName, clone, SC1); assertSampleNotPresent(SC1, scanner, isoScanner, bScanner, csiScanner, oScanner); // configure sampling, however there exist an rfile w/o sample data... so should still see // sample not present exception - updateSamplingConfig(client, tableName, SC1); // create clone with new config @@ -509,4 +611,30 @@ private void check(TreeMap expected, ScannerBase... scanners) { actual.size(), expected.size(), s.getClass().getSimpleName())); } } + + private Set createRanges() { + Set ranges = new HashSet<>(); + + int splits = 10; + + for (int i = 0; i < splits; i++) { + Text start = i > 0 ? new Text(String.format("r_%06d", i * 1000)) : null; + Text end = i < splits - 1 ? new Text(String.format("r_%06d", (i + 1) * 1000)) : null; + ranges.add(new Range(start, end)); + } + + return ranges; + } + + // Create a table and disable compactions. This is important to prevent intermittent + // failures when testing if sampling is configured or not. Some of the tests first + // assert sampling is not available, then configures sampling and tests it still isn't + // available before triggering a compaction to confirm it is now available. Intermittent + // GCs can make these tests non-deterministic when there are a lot of files created + // during the fencing tests. + private void createTable(AccumuloClient client, String tableName, NewTableConfiguration ntc) + throws Exception { + ntc.setProperties(Map.of(Property.TABLE_MAJC_RATIO.getKey(), "9999")); + client.tableOperations().create(tableName, ntc); + } } diff --git a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java index ba5873bed71..5e52066cb14 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanConsistencyIT.java @@ -62,6 +62,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,6 +77,7 @@ * This test verifies that scans will always see data written before the scan started even when * there are concurrent scans, writes, and table operations running. */ +@Disabled // ELASTICITY_TODO public class ScanConsistencyIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(ScanConsistencyIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java index 94bb6fe7a0f..0a9009a8df5 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerConcurrentTabletScanIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.ScanServerIT.createTableAndIngest; import static org.apache.accumulo.test.ScanServerIT.ingest; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -56,7 +55,6 @@ import com.google.common.collect.Iterables; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerConcurrentTabletScanIT extends SharedMiniClusterBase { private static class ScanServerConcurrentTabletScanITConfiguration diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java index 749ebea454c..8d815b96a78 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -87,7 +86,6 @@ import com.google.common.collect.Iterables; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerIT extends SharedMiniClusterBase { private static class ScanServerITConfiguration implements MiniClusterConfigurationCallback { diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java index 054eecfdef5..c6aa4d64a2f 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerIT_NoServers.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.ScanServerIT.createTableAndIngest; import static org.apache.accumulo.test.ScanServerIT.ingest; import static org.apache.accumulo.test.ScanServerIT.setupTableWithHostingMix; @@ -54,7 +53,6 @@ import com.google.common.collect.Iterables; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerIT_NoServers extends SharedMiniClusterBase { // This is the same as ScanServerIT, but without any scan servers running. diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java index b4e1b67c067..a11385d955f 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesCleanIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -40,7 +39,6 @@ import com.google.common.net.HostAndPort; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerMetadataEntriesCleanIT extends SharedMiniClusterBase { @BeforeAll diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java index 6ab7d7c0872..ff5caf6bc6a 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMetadataEntriesIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.ScanServerIT.ingest; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -70,7 +69,6 @@ import com.google.common.net.HostAndPort; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerMetadataEntriesIT extends SharedMiniClusterBase { public static final Logger log = LoggerFactory.getLogger(ScanServerMetadataEntriesIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java index dc810a9ae78..22dd9c52a78 100644 --- a/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java +++ b/test/src/main/java/org/apache/accumulo/test/ScanServerMultipleScansIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test; import static org.apache.accumulo.harness.AccumuloITBase.MINI_CLUSTER_ONLY; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.ScanServerIT.createTableAndIngest; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; @@ -67,7 +66,6 @@ import com.google.common.collect.Iterables; @Tag(MINI_CLUSTER_ONLY) -@Tag(SUNNY_DAY) public class ScanServerMultipleScansIT extends SharedMiniClusterBase { private static final Logger log = LoggerFactory.getLogger(ScanServerMultipleScansIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java index 991e1464102..f3acfa2b53a 100644 --- a/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/SplitRecoveryIT.java @@ -43,8 +43,10 @@ import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class SplitRecoveryIT extends AccumuloClusterHarness { // ELASTICITY_TODO: Confirm still works as intended diff --git a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java index 7c90c74505c..4f9af91004d 100644 --- a/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java +++ b/test/src/main/java/org/apache/accumulo/test/UnusedWALIT.java @@ -45,12 +45,14 @@ import org.apache.accumulo.test.functional.ConfigurableMacBase; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; // When reviewing the changes for ACCUMULO-3423, kturner suggested // "tablets will now have log references that contain no data, // so it may be marked with 3 WALs, the first with data, the 2nd without, a 3rd with data. // It would be useful to have an IT that will test this situation. +@Disabled // ELASTICITY_TODO public class UnusedWALIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java index 42b06dac615..7bddfa9c66d 100644 --- a/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/VerifySerialRecoveryIT.java @@ -48,8 +48,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class VerifySerialRecoveryIT extends ConfigurableMacBase { private static final byte[] HEXCHARS = {0x30, 0x31, 0x32, 0x33, 0x34, 0x35, 0x36, 0x37, 0x38, diff --git a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java index 4f231deb6a0..bc448f2314d 100644 --- a/test/src/main/java/org/apache/accumulo/test/VolumeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/VolumeIT.java @@ -36,6 +36,7 @@ import java.util.Map.Entry; import java.util.SortedSet; import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import org.apache.accumulo.core.Constants; @@ -64,15 +65,20 @@ import org.apache.accumulo.core.metadata.MetadataTable; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.DataFileValue; import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.DataFileColumnFamily; import org.apache.accumulo.core.security.Authorizations; +import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; +import org.apache.accumulo.server.ServerContext; import org.apache.accumulo.server.init.Initialize; import org.apache.accumulo.server.log.WalStateManager; import org.apache.accumulo.server.log.WalStateManager.WalMarkerException; import org.apache.accumulo.server.log.WalStateManager.WalState; +import org.apache.accumulo.server.security.SystemCredentials; import org.apache.accumulo.server.util.Admin; import org.apache.accumulo.test.functional.ConfigurableMacBase; +import org.apache.accumulo.test.util.FileMetadataUtil; import org.apache.commons.configuration2.PropertiesConfiguration; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; @@ -81,10 +87,12 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; import org.apache.zookeeper.KeeperException.NoNodeException; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +@Disabled // ELASTICITY_TODO public class VolumeIT extends ConfigurableMacBase { private File volDirBase; @@ -278,6 +286,11 @@ private void writeData(String tableName, AccumuloClient client) throws AccumuloE private void verifyVolumesUsed(AccumuloClient client, String tableName, boolean shouldExist, Path... paths) throws Exception { + verifyVolumesUsed(client, tableName, shouldExist, false, paths); + } + + private void verifyVolumesUsed(AccumuloClient client, String tableName, boolean shouldExist, + boolean rangedFiles, Path... paths) throws Exception { if (!client.tableOperations().exists(tableName)) { assertFalse(shouldExist); @@ -346,7 +359,10 @@ private void verifyVolumesUsed(AccumuloClient client, String tableName, boolean sum += count; } - assertEquals(100, sum); + // When ranged files exist we there should be twice as many + // as the test split each file into 2 + int expectedCount = rangedFiles ? 200 : 100; + assertEquals(expectedCount, sum); } } @@ -392,7 +408,8 @@ public void testRemoveVolumes() throws Exception { } } - private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) throws Exception { + private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown, boolean rangedFiles) + throws Exception { String[] tableNames = getUniqueNames(3); verifyVolumesUsed(client, tableNames[0], false, v1, v2); @@ -403,6 +420,13 @@ private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) thr writeData(tableNames[1], c2); } + // If flag is true then for each file split and create two files + // to verify volume replacement works on files with ranges + if (rangedFiles) { + splitFilesWithRange(client, tableNames[0]); + splitFilesWithRange(client, tableNames[1]); + } + if (cleanShutdown) { assertEquals(0, cluster.exec(Admin.class, "stopAll").getProcess().waitFor()); } @@ -428,15 +452,16 @@ private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) thr // start cluster and verify that volumes were replaced cluster.start(); - verifyVolumesUsed(client, tableNames[0], true, v8, v9); - verifyVolumesUsed(client, tableNames[1], true, v8, v9); + verifyVolumesUsed(client, tableNames[0], true, rangedFiles, v8, v9); + verifyVolumesUsed(client, tableNames[1], true, rangedFiles, v8, v9); // verify writes to new dir client.tableOperations().compact(tableNames[0], null, null, true, true); client.tableOperations().compact(tableNames[1], null, null, true, true); - verifyVolumesUsed(client, tableNames[0], true, v8, v9); - verifyVolumesUsed(client, tableNames[1], true, v8, v9); + // Always pass false for ranged files as compaction will clean them up if exist + verifyVolumesUsed(client, tableNames[0], true, false, v8, v9); + verifyVolumesUsed(client, tableNames[1], true, false, v8, v9); client.tableOperations().compact(RootTable.NAME, new CompactionConfig().setWait(true)); @@ -465,14 +490,28 @@ private void testReplaceVolume(AccumuloClient client, boolean cleanShutdown) thr @Test public void testCleanReplaceVolumes() throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { - testReplaceVolume(client, true); + testReplaceVolume(client, true, false); } } @Test public void testDirtyReplaceVolumes() throws Exception { try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { - testReplaceVolume(client, false); + testReplaceVolume(client, false, false); + } + } + + @Test + public void testCleanReplaceVolumesWithRangedFiles() throws Exception { + try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + testReplaceVolume(client, true, true); + } + } + + @Test + public void testDirtyReplaceVolumesWithRangedFiles() throws Exception { + try (AccumuloClient client = Accumulo.newClient().from(getClientProperties()).build()) { + testReplaceVolume(client, false, true); } } @@ -488,4 +527,44 @@ private void updateConfig(Consumer updater) throws Exce config.write(out); } } + + // Go through each tablet file in metadata and split the files into two files + // by adding two new entries that covers half of the file. This will test that + // files with ranges work properly with volume replacement + private void splitFilesWithRange(AccumuloClient client, String tableName) throws Exception { + client.securityOperations().grantTablePermission(cluster.getConfig().getRootUserName(), + MetadataTable.NAME, TablePermission.WRITE); + final ServerContext ctx = getServerContext(); + ctx.setCredentials(new SystemCredentials(client.instanceOperations().getInstanceId(), "root", + new PasswordToken(ROOT_PASSWORD))); + + AtomicInteger i = new AtomicInteger(); + FileMetadataUtil.mutateTabletFiles(ctx, tableName, null, null, (tm, mutator, file, value) -> { + i.incrementAndGet(); + + // Create a mutation to delete the existing file metadata entry with infinite range + mutator.deleteFile(file); + + // Find the midpoint and create two new files, each with a range covering half the file + Text tabletMidPoint = getTabletMidPoint(tm.getExtent().endRow()); + // Handle edge case for last tablet + if (tabletMidPoint == null) { + tabletMidPoint = new Text( + String.format("%06d", Integer.parseInt(tm.getExtent().prevEndRow().toString()) + 50)); + } + + final DataFileValue newValue = new DataFileValue(Integer.max(1, (int) (value.getSize() / 2)), + Integer.max(1, (int) (value.getNumEntries() / 2))); + mutator.putFile(StoredTabletFile.of(file.getPath(), + new Range(tm.getExtent().prevEndRow(), tabletMidPoint)), newValue); + mutator.putFile( + StoredTabletFile.of(file.getPath(), new Range(tabletMidPoint, tm.getExtent().endRow())), + newValue); + }); + } + + private static Text getTabletMidPoint(Text row) { + return row != null ? new Text(String.format("%06d", Integer.parseInt(row.toString()) - 50)) + : null; + } } diff --git a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java index 55d3c7e1a89..1c0b33ce378 100644 --- a/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java +++ b/test/src/main/java/org/apache/accumulo/test/WaitForBalanceIT.java @@ -41,8 +41,10 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.test.functional.ConfigurableMacBase; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class WaitForBalanceIT extends ConfigurableMacBase { private static final int NUM_SPLITS = 50; diff --git a/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java b/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java index 7c5324c80bd..5a201315088 100644 --- a/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java +++ b/test/src/main/java/org/apache/accumulo/test/WriteAfterCloseIT.java @@ -51,6 +51,7 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class WriteAfterCloseIT extends AccumuloClusterHarness { @@ -104,6 +105,7 @@ public void testWriteAfterCloseLogicalTime() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void testWriteAfterCloseKillTservers() throws Exception { runTest(TimeType.MILLIS, true, 0, false); } diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java index 60ede3dff79..4634603e7d4 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionConfigChangeIT.java @@ -35,8 +35,10 @@ import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.test.functional.SlowIterator; import org.apache.accumulo.test.util.Wait; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class CompactionConfigChangeIT extends AccumuloClusterHarness { public static long countFiles(AccumuloClient client, String table, String fileNamePrefix) diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java index 1832117ce5b..3ae95fbf20a 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionExecutorIT.java @@ -74,8 +74,10 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class CompactionExecutorIT extends SharedMiniClusterBase { public static class TestPlanner implements CompactionPlanner { diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java deleted file mode 100644 index 0b4280f8d10..00000000000 --- a/test/src/main/java/org/apache/accumulo/test/compaction/CompactionRateLimitingIT.java +++ /dev/null @@ -1,110 +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.test.compaction; - -import static org.apache.accumulo.core.util.LazySingletons.RANDOM; -import static org.junit.jupiter.api.Assertions.assertTrue; - -import java.util.Map; - -import org.apache.accumulo.core.client.AccumuloClient; -import org.apache.accumulo.core.client.BatchWriter; -import org.apache.accumulo.core.client.admin.NewTableConfiguration; -import org.apache.accumulo.core.client.security.tokens.PasswordToken; -import org.apache.accumulo.core.conf.Property; -import org.apache.accumulo.core.data.Mutation; -import org.apache.accumulo.core.spi.compaction.DefaultCompactionPlanner; -import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; -import org.apache.accumulo.test.functional.ConfigurableMacBase; -import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Test; - -public class CompactionRateLimitingIT extends ConfigurableMacBase { - public static final long BYTES_TO_WRITE = 10 * 1024 * 1024; - public static final long RATE = 1 * 1024 * 1024; - - protected Property getThroughputProp() { - return Property.TSERV_COMPACTION_SERVICE_DEFAULT_RATE_LIMIT; - } - - @Override - public void configure(MiniAccumuloConfigImpl cfg, Configuration fsConf) { - cfg.setProperty(getThroughputProp(), RATE + "B"); - cfg.setProperty(Property.TABLE_MAJC_RATIO, "20"); - cfg.setProperty(Property.TABLE_FILE_COMPRESSION_TYPE, "none"); - - cfg.setProperty("tserver.compaction.major.service.test.rate.limit", RATE + "B"); - cfg.setProperty("tserver.compaction.major.service.test.planner", - DefaultCompactionPlanner.class.getName()); - cfg.setProperty("tserver.compaction.major.service.test.planner.opts.executors", - "[{'name':'all','numThreads':2}]".replaceAll("'", "\"")); - - } - - @Test - public void majorCompactionsAreRateLimited() throws Exception { - long bytesWritten = 0; - String[] tableNames = getUniqueNames(1); - - try (AccumuloClient client = - getCluster().createAccumuloClient("root", new PasswordToken(ROOT_PASSWORD))) { - - for (int i = 0; i < tableNames.length; i++) { - String tableName = tableNames[i]; - - NewTableConfiguration ntc = new NewTableConfiguration(); - if (i == 1) { - ntc.setProperties(Map.of("table.compaction.dispatcher.opts.service", "test")); - } - - client.tableOperations().create(tableName, ntc); - try (BatchWriter bw = client.createBatchWriter(tableName)) { - while (bytesWritten < BYTES_TO_WRITE) { - byte[] rowKey = new byte[32]; - RANDOM.get().nextBytes(rowKey); - - byte[] qual = new byte[32]; - RANDOM.get().nextBytes(qual); - - byte[] value = new byte[1024]; - RANDOM.get().nextBytes(value); - - Mutation m = new Mutation(rowKey); - m.put(new byte[0], qual, value); - bw.addMutation(m); - - bytesWritten += rowKey.length + qual.length + value.length; - } - } - - client.tableOperations().flush(tableName, null, null, true); - - long compactionStart = System.currentTimeMillis(); - client.tableOperations().compact(tableName, null, null, false, true); - long duration = System.currentTimeMillis() - compactionStart; - // The rate will be "bursty", try to account for that by taking 80% of the expected rate - // (allow for 20% under the maximum expected duration) - String message = String.format( - "Expected a compaction rate of no more than %,d bytes/sec, but saw a rate of %,f bytes/sec", - (int) (0.8d * RATE), 1000.0 * bytesWritten / duration); - assertTrue(duration > 1000L * 0.8 * BYTES_TO_WRITE / RATE, message); - } - } - } -} diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java index 36215f46e1c..140f746b99b 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction4_IT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.compaction; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.createTable; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -44,10 +43,8 @@ import org.apache.accumulo.test.functional.ErrorThrowingIterator; import org.apache.accumulo.test.functional.ReadWriteIT; import org.apache.hadoop.conf.Configuration; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -@Tag(SUNNY_DAY) public class ExternalCompaction4_IT extends AccumuloClusterHarness { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java index 3da31e2f2cb..7993703c301 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompactionMetricsIT.java @@ -52,8 +52,10 @@ import org.apache.hadoop.conf.Configuration; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class ExternalCompactionMetricsIT extends SharedMiniClusterBase { public static class ExternalCompactionMetricsITConfig diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java index c91c6e2d7c8..af13e5e576b 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_1_IT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.compaction; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP1; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP2; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP3; @@ -80,12 +79,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import com.google.common.base.Preconditions; -@Tag(SUNNY_DAY) public class ExternalCompaction_1_IT extends SharedMiniClusterBase { public static class ExternalCompaction1Config implements MiniClusterConfigurationCallback { diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java index 273fa854469..f532451c187 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_2_IT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.compaction; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP1; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP3; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP4; @@ -66,11 +65,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.slf4j.LoggerFactory; -@Tag(SUNNY_DAY) public class ExternalCompaction_2_IT extends SharedMiniClusterBase { public static class ExternalCompaction2Config implements MiniClusterConfigurationCallback { diff --git a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java index c48f522b336..52da17e0095 100644 --- a/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java +++ b/test/src/main/java/org/apache/accumulo/test/compaction/ExternalCompaction_3_IT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.compaction; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.GROUP2; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.compact; import static org.apache.accumulo.test.compaction.ExternalCompactionTestUtils.confirmCompactionCompleted; @@ -60,12 +59,10 @@ import org.apache.thrift.transport.TTransportException; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Disabled; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import com.google.common.net.HostAndPort; -@Tag(SUNNY_DAY) public class ExternalCompaction_3_IT extends SharedMiniClusterBase { public static class ExternalCompaction3Config implements MiniClusterConfigurationCallback { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java index 694f4ee755f..b36f3aef928 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AccumuloClientIT.java @@ -44,6 +44,7 @@ import org.apache.accumulo.core.singletons.SingletonManager.Mode; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class AccumuloClientIT extends AccumuloClusterHarness { @@ -140,6 +141,7 @@ public void testAccumuloClientBuilder() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void testClose() throws Exception { String tableName = getUniqueNames(1)[0]; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java index 81883b0e2b1..3c59eac3372 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AmpleConditionalWriterIT.java @@ -23,9 +23,9 @@ import static org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.ServerColumnFamily.SELECTED_COLUMN; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.COMPACTED; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES; +import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FLUSH_ID; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOADED; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOCATION; -import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.SELECTED; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.TIME; import static org.apache.accumulo.core.util.LazySingletons.GSON; @@ -33,6 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; import java.util.ArrayList; import java.util.Collection; @@ -201,7 +202,7 @@ public void testFiles() throws Exception { var tm1 = TabletMetadata.builder(e1).putFile(stf1, dfv).putFile(stf2, dfv).putFile(stf3, dfv) .build(); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, PREV_ROW, FILES) + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, FILES) .putFile(stf4, new DataFileValue(0, 0)).submit(tm -> false); var results = ctmi.process(); assertEquals(Status.REJECTED, results.get(e1).getStatus()); @@ -337,15 +338,15 @@ public void testSelectedFiles() throws Exception { // simulate a compaction where the tablet location is not set var ctmi = new ConditionalTabletsMutatorImpl(context); var tm1 = TabletMetadata.builder(e1).build(FILES, SELECTED); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, PREV_ROW, FILES) - .putFile(stf1, dfv).putFile(stf2, dfv).putFile(stf3, dfv).submit(tm -> false); + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, FILES).putFile(stf1, dfv) + .putFile(stf2, dfv).putFile(stf3, dfv).submit(tm -> false); var results = ctmi.process(); assertEquals(Status.ACCEPTED, results.get(e1).getStatus()); assertEquals(Set.of(stf1, stf2, stf3), context.getAmple().readTablet(e1).getFiles()); ctmi = new ConditionalTabletsMutatorImpl(context); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, PREV_ROW, FILES, SELECTED) + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm1, FILES, SELECTED) .putSelectedFiles(new SelectedFiles(Set.of(stf1, stf2, stf3), true, 2L)) .submit(tm -> false); results = ctmi.process(); @@ -357,7 +358,7 @@ public void testSelectedFiles() throws Exception { var tm2 = TabletMetadata.builder(e1).putFile(stf1, dfv).putFile(stf2, dfv).putFile(stf3, dfv) .build(SELECTED); ctmi = new ConditionalTabletsMutatorImpl(context); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm2, PREV_ROW, FILES, SELECTED) + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm2, FILES, SELECTED) .putSelectedFiles(new SelectedFiles(Set.of(stf1, stf2, stf3), true, 2L)) .submit(tm -> false); results = ctmi.process(); @@ -380,7 +381,7 @@ public void testSelectedFiles() throws Exception { var tm3 = TabletMetadata.builder(e1).putFile(stf1, dfv).putFile(stf2, dfv).putFile(stf3, dfv) .putSelectedFiles(selectedFiles).build(); ctmi = new ConditionalTabletsMutatorImpl(context); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm3, PREV_ROW, FILES, SELECTED) + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm3, FILES, SELECTED) .deleteSelectedFiles().submit(tm -> false); results = ctmi.process(); assertEquals(Status.REJECTED, results.get(e1).getStatus()); @@ -393,7 +394,7 @@ public void testSelectedFiles() throws Exception { var tm5 = TabletMetadata.builder(e1).putFile(stf1, dfv).putFile(stf2, dfv).putFile(stf3, dfv) .putSelectedFiles(new SelectedFiles(Set.of(stf1, stf2, stf3), true, 2L)).build(); ctmi = new ConditionalTabletsMutatorImpl(context); - ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm5, PREV_ROW, FILES, SELECTED) + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tm5, FILES, SELECTED) .deleteSelectedFiles().submit(tm -> false); results = ctmi.process(); assertEquals(Status.ACCEPTED, results.get(e1).getStatus()); @@ -525,7 +526,7 @@ public static String createSelectedFilesJson(Long txid, boolean selAll, } @Test - public void testMultipleExtents() throws Exception { + public void testMultipleExtents() { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var ts1 = new TServerInstance("localhost:9997", 5000L); var ts2 = new TServerInstance("localhost:9997", 6000L); @@ -576,7 +577,7 @@ public void testMultipleExtents() throws Exception { } @Test - public void testOperations() throws Exception { + public void testOperations() { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { var context = cluster.getServerContext(); @@ -691,7 +692,7 @@ public void testCompacted() { } @Test - public void testRootTabletUpdate() throws Exception { + public void testRootTabletUpdate() { var context = cluster.getServerContext(); var rootMeta = context.getAmple().readTablet(RootTable.EXTENT); @@ -754,4 +755,48 @@ public void testTime() { } } } + + @Test + public void testFlushId() { + try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { + var context = cluster.getServerContext(); + + assertTrue(context.getAmple().readTablet(e1).getFlushId().isEmpty()); + + var ctmi = new ConditionalTabletsMutatorImpl(context); + + var tabletMeta1 = TabletMetadata.builder(e1).putFlushId(42L).build(); + assertTrue(tabletMeta1.getFlushId().isPresent()); + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tabletMeta1, FLUSH_ID) + .putFlushId(43L).submit(tabletMetadata -> tabletMetadata.getFlushId().orElse(-1) == 43L); + var results = ctmi.process(); + assertEquals(Status.REJECTED, results.get(e1).getStatus()); + assertTrue(context.getAmple().readTablet(e1).getFlushId().isEmpty()); + + ctmi = new ConditionalTabletsMutatorImpl(context); + var tabletMeta2 = TabletMetadata.builder(e1).build(FLUSH_ID); + assertFalse(tabletMeta2.getFlushId().isPresent()); + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tabletMeta2, FLUSH_ID) + .putFlushId(43L).submit(tabletMetadata -> tabletMetadata.getFlushId().orElse(-1) == 43L); + results = ctmi.process(); + assertEquals(Status.ACCEPTED, results.get(e1).getStatus()); + assertEquals(43L, context.getAmple().readTablet(e1).getFlushId().getAsLong()); + + ctmi = new ConditionalTabletsMutatorImpl(context); + var tabletMeta3 = TabletMetadata.builder(e1).putFlushId(43L).build(); + assertTrue(tabletMeta1.getFlushId().isPresent()); + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tabletMeta3, FLUSH_ID) + .putFlushId(44L).submit(tabletMetadata -> tabletMetadata.getFlushId().orElse(-1) == 44L); + results = ctmi.process(); + assertEquals(Status.ACCEPTED, results.get(e1).getStatus()); + assertEquals(44L, context.getAmple().readTablet(e1).getFlushId().getAsLong()); + + ctmi = new ConditionalTabletsMutatorImpl(context); + ctmi.mutateTablet(e1).requireAbsentOperation().requireSame(tabletMeta3, FLUSH_ID) + .putFlushId(45L).submit(tabletMetadata -> tabletMetadata.getFlushId().orElse(-1) == 45L); + results = ctmi.process(); + assertEquals(Status.REJECTED, results.get(e1).getStatus()); + assertEquals(44L, context.getAmple().readTablet(e1).getFlushId().getAsLong()); + } + } } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java index 6e326de445a..e742cbbde43 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/AssignLocationModeIT.java @@ -37,8 +37,10 @@ import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class AssignLocationModeIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java index dc68fb95452..6b184c302c7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BigRootTabletIT.java @@ -32,8 +32,10 @@ import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class BigRootTabletIT extends AccumuloClusterHarness { // ACCUMULO-542: A large root tablet will fail to load if it does't fit in the tserver scan // buffers diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java index b4e933c3845..a70dfddfef7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkFailureIT.java @@ -83,12 +83,14 @@ import org.apache.thrift.TServiceClient; import org.apache.thrift.transport.TTransportException; import org.apache.zookeeper.KeeperException; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.net.HostAndPort; +@Disabled // ELASTICITY_TODO public class BulkFailureIT extends AccumuloClusterHarness { private static final Logger LOG = LoggerFactory.getLogger(BulkFailureIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java index c22617fd8a5..3e63bed845c 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/BulkNewIT.java @@ -21,7 +21,6 @@ import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.FILES; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.LOADED; import static org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType.PREV_ROW; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -86,12 +85,10 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -@Tag(SUNNY_DAY) public class BulkNewIT extends SharedMiniClusterBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java index af994e608c1..e0ab80f3ce5 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CleanTmpIT.java @@ -44,12 +44,14 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.collect.Iterators; +@Disabled // ELASTICITY_TODO public class CleanTmpIT extends ConfigurableMacBase { private static final Logger log = LoggerFactory.getLogger(CleanTmpIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java index 6af6bc19d6d..b45c0337bd2 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/CompactionIT.java @@ -21,7 +21,6 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toList; import static java.util.stream.Collectors.toSet; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -81,7 +80,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -90,7 +88,6 @@ import com.google.common.base.Supplier; import com.google.common.collect.Iterators; -@Tag(SUNNY_DAY) public class CompactionIT extends AccumuloClusterHarness { public static class TestFilter extends Filter { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java index e8b04be8f2c..33d360c5fed 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/DeleteRowsSplitIT.java @@ -39,11 +39,13 @@ import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; // attempt to reproduce ACCUMULO-315 +@Disabled // ELASTICITY_TODO public class DeleteRowsSplitIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(DeleteRowsSplitIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java index a2683c34f13..9ffedb48e14 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/DurabilityIT.java @@ -38,12 +38,14 @@ import org.apache.accumulo.miniclusterImpl.ProcessReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterators; @Tag(MINI_CLUSTER_ONLY) +@Disabled // ELASTICITY_TODO public class DurabilityIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java index 9cdd9d80947..2283132c3f7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FateConcurrencyIT.java @@ -57,6 +57,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,6 +73,7 @@ * (original) and additional method without. * */ +@Disabled // ELASTICITY_TODO public class FateConcurrencyIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(FateConcurrencyIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java index 47fd8da48e4..bf739d5e720 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FileMetadataIT.java @@ -57,6 +57,7 @@ import org.apache.accumulo.test.VerifyIngest.VerifyParams; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -351,6 +352,7 @@ public void splitsRangeTest() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void splitsWithExistingRangesTest() throws Exception { ServerContext ctx = getCluster().getServerContext(); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/FlushNoFileIT.java b/test/src/main/java/org/apache/accumulo/test/functional/FlushNoFileIT.java index 0c39306daf5..75aaa17a1a2 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/FlushNoFileIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/FlushNoFileIT.java @@ -47,6 +47,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.harness.AccumuloClusterHarness; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterables; @@ -54,6 +55,7 @@ /** * Tests that Accumulo will flush but not create a file that has 0 entries. */ +@Disabled // ELASTICITY_TODO public class FlushNoFileIT extends AccumuloClusterHarness { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java index c7cb6229273..f34dbf9f370 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorIT.java @@ -80,6 +80,7 @@ import org.apache.hadoop.io.Text; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -130,6 +131,7 @@ private void killMacGc() throws ProcessNotFoundException, InterruptedException, } @Test + @Disabled // ELASTICITY_TODO public void gcTest() throws Exception { killMacGc(); final String table = "test_ingest"; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java index d98a7b76957..f327761da28 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashDefaultIT.java @@ -36,9 +36,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; // verify trash is not used with Hadoop defaults, since Trash is not enabled by default +@Disabled // ELASTICITY_TODO public class GarbageCollectorTrashDefaultIT extends GarbageCollectorTrashBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java index ca5979793c3..3d6d06e5971 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledIT.java @@ -36,9 +36,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; // verify that trash is used if Hadoop is configured to use it +@Disabled // ELASTICITY_TODO public class GarbageCollectorTrashEnabledIT extends GarbageCollectorTrashBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java index 9d5f06b9942..7aafb3cfcfc 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/GarbageCollectorTrashEnabledWithCustomPolicyIT.java @@ -40,9 +40,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.TrashPolicyDefault; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; // verify that trash is used if Hadoop is configured to use it and that using a custom policy works +@Disabled // ELASTICITY_TODO public class GarbageCollectorTrashEnabledWithCustomPolicyIT extends GarbageCollectorTrashBase { public static class NoFlushFilesInTrashPolicy extends TrashPolicyDefault { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java index 0f82311d7a5..b7542800dec 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ManagerAssignmentIT.java @@ -84,11 +84,13 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterables; import com.google.common.net.HostAndPort; +@Disabled // ELASTICITY_TODO public class ManagerAssignmentIT extends SharedMiniClusterBase { @BeforeAll diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java index bea98486358..a0382ca2cb5 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/MemoryStarvedMajCIT.java @@ -47,8 +47,10 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class MemoryStarvedMajCIT extends SharedMiniClusterBase { public static class MemoryStarvedITConfiguration implements MiniClusterConfigurationCallback { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java index 8672f4e0380..94e59279277 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/MergeIT.java @@ -19,7 +19,6 @@ package org.apache.accumulo.test.functional; import static java.nio.charset.StandardCharsets.UTF_8; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.apache.accumulo.test.util.FileMetadataUtil.printAndVerifyFileMetadata; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -66,14 +65,12 @@ import org.apache.accumulo.test.VerifyIngest; import org.apache.accumulo.test.VerifyIngest.VerifyParams; import org.apache.hadoop.io.Text; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.google.common.collect.Iterables; -@Tag(SUNNY_DAY) public class MergeIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(MergeIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java index 4827d5a64d4..cf597b5709a 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/MetadataMaxFilesIT.java @@ -41,8 +41,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class MetadataMaxFilesIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java index b18e5912706..6578927ea24 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/OnDemandTabletUnloadingIT.java @@ -18,7 +18,6 @@ */ package org.apache.accumulo.test.functional; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -60,10 +59,8 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -@Tag(SUNNY_DAY) public class OnDemandTabletUnloadingIT extends SharedMiniClusterBase { private static final int managerTabletGroupWatcherInterval = 5; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java index bf5aaab74e7..b8147691d80 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/RegexGroupBalanceIT.java @@ -45,11 +45,13 @@ import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.commons.lang3.mutable.MutableInt; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import com.google.common.collect.HashBasedTable; import com.google.common.collect.Table; +@Disabled // ELASTICITY_TODO public class RegexGroupBalanceIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java index 5e952629945..adbd878e6a4 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/RestartIT.java @@ -51,6 +51,7 @@ import org.apache.hadoop.fs.RawLocalFileSystem; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -116,6 +117,7 @@ public void restartManager() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void restartManagerRecovery() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; @@ -201,6 +203,7 @@ public void restartManagerSplit() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void killedTabletServer() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; @@ -215,6 +218,7 @@ public void killedTabletServer() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void killedTabletServer2() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { final String[] names = getUniqueNames(2); @@ -231,6 +235,7 @@ public void killedTabletServer2() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void killedTabletServerDuringShutdown() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { String tableName = getUniqueNames(1)[0]; diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java index 572b1d0b536..f63bf24358e 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SessionDurabilityIT.java @@ -43,10 +43,12 @@ import org.apache.accumulo.miniclusterImpl.ProcessReference; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import com.google.common.collect.Iterators; +@Disabled // ELASTICITY_TODO public class SessionDurabilityIT extends ConfigurableMacBase { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java index 60ab7dd662a..07f389ae3ec 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/ShutdownIT.java @@ -33,6 +33,7 @@ import org.apache.accumulo.test.TestIngest; import org.apache.accumulo.test.TestRandomDeletes; import org.apache.accumulo.test.VerifyIngest; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class ShutdownIT extends ConfigurableMacBase { @@ -106,6 +107,7 @@ public void stopDuringStart() throws Exception { } @Test + @Disabled // ELASTICITY_TODO public void adminStop() throws Exception { try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) { runAdminStopTest(c, cluster); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java index eb28de8e12b..852d2c2ec3e 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitIT.java @@ -21,7 +21,6 @@ import static java.util.Collections.singletonMap; import static java.util.concurrent.TimeUnit.SECONDS; import static org.apache.accumulo.core.util.LazySingletons.RANDOM; -import static org.apache.accumulo.harness.AccumuloITBase.SUNNY_DAY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -75,7 +74,6 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,7 +82,6 @@ import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -@Tag(SUNNY_DAY) public class SplitIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(SplitIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java index c783e8156e2..ec1b65a86f1 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SplitRecoveryIT.java @@ -78,10 +78,12 @@ import org.apache.accumulo.server.zookeeper.TransactionWatcher; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.Text; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +@Disabled // ELASTICITY_TODO public class SplitRecoveryIT extends ConfigurableMacBase { // ELASTICITY_TODO: This functionality needs to be adpated to work on upgrade diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java index 6e422ebbe46..d8709e4502d 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SslWithClientAuthIT.java @@ -24,6 +24,7 @@ import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.miniclusterImpl.MiniAccumuloConfigImpl; import org.apache.hadoop.conf.Configuration; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; /** @@ -60,6 +61,7 @@ public void concurrency() throws Exception { @Override @Test + @Disabled // ELASTICITY_TODO public void adminStop() throws Exception { super.adminStop(); } diff --git a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java index d3fab0a6abf..2621abcd311 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/SummaryIT.java @@ -88,6 +88,7 @@ import org.apache.hadoop.io.Text; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; public class SummaryIT extends SharedMiniClusterBase { @@ -139,6 +140,7 @@ private void addSplits(final String table, AccumuloClient c, String... splits) } @Test + @Disabled // ELASTICITY_TODO public void basicSummaryTest() throws Exception { final String table = getUniqueNames(1)[0]; try (AccumuloClient c = Accumulo.newClient().from(getClientProps()).build()) { diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java index 1bbceac3ece..9a090da29b7 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogEncryptedIT.java @@ -31,10 +31,12 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@Disabled // ELASTICITY_TODO public class WriteAheadLogEncryptedIT extends AccumuloClusterHarness { private static final Logger log = LoggerFactory.getLogger(WriteAheadLogEncryptedIT.class); diff --git a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java index ef14b75d81e..3f58bb88ad3 100644 --- a/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java +++ b/test/src/main/java/org/apache/accumulo/test/functional/WriteAheadLogIT.java @@ -29,8 +29,10 @@ import org.apache.accumulo.test.VerifyIngest.VerifyParams; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.RawLocalFileSystem; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; +@Disabled // ELASTICITY_TODO public class WriteAheadLogIT extends AccumuloClusterHarness { @Override diff --git a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java b/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java deleted file mode 100644 index 1806f4c6382..00000000000 --- a/test/src/main/java/org/apache/accumulo/test/manager/MergeStateIT.java +++ /dev/null @@ -1,247 +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.test.manager; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.TreeSet; - -import org.apache.accumulo.core.client.Accumulo; -import org.apache.accumulo.core.client.AccumuloClient; -import org.apache.accumulo.core.client.BatchDeleter; -import org.apache.accumulo.core.client.BatchWriter; -import org.apache.accumulo.core.client.MutationsRejectedException; -import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.client.admin.NewTableConfiguration; -import org.apache.accumulo.core.client.admin.TabletHostingGoal; -import org.apache.accumulo.core.data.Mutation; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.TableId; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.dataImpl.KeyExtent; -import org.apache.accumulo.core.manager.state.TabletManagement; -import org.apache.accumulo.core.manager.state.TabletManagement.ManagementAction; -import org.apache.accumulo.core.manager.thrift.ManagerState; -import org.apache.accumulo.core.metadata.MetadataTable; -import org.apache.accumulo.core.metadata.TServerInstance; -import org.apache.accumulo.core.metadata.TabletState; -import org.apache.accumulo.core.metadata.schema.Ample.DataLevel; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.CurrentLocationColumnFamily; -import org.apache.accumulo.core.metadata.schema.MetadataSchema.TabletsSection.TabletColumnFamily; -import org.apache.accumulo.core.metadata.schema.TabletMetadata; -import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; -import org.apache.accumulo.core.metadata.schema.TabletMetadata.Location; -import org.apache.accumulo.core.security.Authorizations; -import org.apache.accumulo.core.security.TablePermission; -import org.apache.accumulo.core.tabletserver.log.LogEntry; -import org.apache.accumulo.manager.state.MergeStats; -import org.apache.accumulo.server.ServerContext; -import org.apache.accumulo.server.manager.state.Assignment; -import org.apache.accumulo.server.manager.state.CurrentState; -import org.apache.accumulo.server.manager.state.MergeInfo; -import org.apache.accumulo.server.manager.state.MergeState; -import org.apache.accumulo.server.manager.state.TabletStateStore; -import org.apache.accumulo.test.functional.ConfigurableMacBase; -import org.apache.hadoop.io.Text; -import org.junit.jupiter.api.Test; - -import com.google.common.net.HostAndPort; - -public class MergeStateIT extends ConfigurableMacBase { - - private static class MockCurrentState implements CurrentState { - - TServerInstance someTServer = - new TServerInstance(HostAndPort.fromParts("127.0.0.1", 1234), 0x123456); - MergeInfo mergeInfo; - - MockCurrentState(MergeInfo info) { - this.mergeInfo = info; - } - - @Override - public Set onlineTables() { - return Collections.singleton(TableId.of("t")); - } - - @Override - public Set onlineTabletServers() { - return Collections.singleton(someTServer); - } - - @Override - public Map> tServerResourceGroups() { - return new HashMap<>(); - } - - @Override - public Collection merges() { - return Collections.singleton(mergeInfo); - } - - @Override - public Set migrationsSnapshot() { - return Collections.emptySet(); - } - - @Override - public ManagerState getManagerState() { - return ManagerState.NORMAL; - } - - @Override - public Map> getCompactionHints() { - return Map.of(); - } - - @Override - public Set shutdownServers() { - return Collections.emptySet(); - } - - } - - private static void update(AccumuloClient c, Mutation m) - throws TableNotFoundException, MutationsRejectedException { - try (BatchWriter bw = c.createBatchWriter(MetadataTable.NAME)) { - bw.addMutation(m); - } - } - - @Test - public void test() throws Exception { - ServerContext context = getServerContext(); - try (AccumuloClient accumuloClient = Accumulo.newClient().from(getClientProperties()).build()) { - accumuloClient.securityOperations().grantTablePermission(accumuloClient.whoami(), - MetadataTable.NAME, TablePermission.WRITE); - BatchWriter bw = accumuloClient.createBatchWriter(MetadataTable.NAME); - - TreeSet splits = new TreeSet<>(); - splits.add(new Text("a")); - splits.add(new Text("e")); - splits.add(new Text("j")); - splits.add(new Text("o")); - splits.add(new Text("t")); - splits.add(new Text("z")); - NewTableConfiguration ntc = new NewTableConfiguration(); - ntc.withSplits(splits); - accumuloClient.tableOperations().create("merge_test_table"); - TableId tableId = - TableId.of(accumuloClient.tableOperations().tableIdMap().get("merge_test_table")); - - Text pr = null; - for (Text split : splits) { - Mutation prevRow = - TabletColumnFamily.createPrevRowMutation(new KeyExtent(tableId, split, pr)); - prevRow.put(CurrentLocationColumnFamily.NAME, new Text("123456"), - new Value("127.0.0.1:1234")); - bw.addMutation(prevRow); - pr = split; - } - bw.close(); - - // Read out the TabletLocationStates - MockCurrentState state = - new MockCurrentState(new MergeInfo(new KeyExtent(tableId, new Text("p"), new Text("e")), - MergeInfo.Operation.MERGE)); - - // Verify the tablet state: hosted, and count - TabletStateStore metaDataStateStore = - TabletStateStore.getStoreForLevel(DataLevel.USER, context, state); - int count = 0; - for (TabletManagement mti : metaDataStateStore) { - if (mti != null) { - assertEquals(1, mti.actions.size()); - assertEquals(ManagementAction.NEEDS_LOCATION_UPDATE, mti.getActions().iterator().next()); - count++; - } - } - assertEquals(6, count); - - // Create the hole - // Split the tablet at one end of the range - Mutation m = TabletColumnFamily - .createPrevRowMutation(new KeyExtent(tableId, new Text("t"), new Text("p"))); - TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5")); - TabletColumnFamily.OLD_PREV_ROW_COLUMN.put(m, - TabletColumnFamily.encodePrevEndRow(new Text("o"))); - update(accumuloClient, m); - - // ELASTICITY_TODO: Tried to fix this up, not sure how this works - - // do the state check - MergeStats stats = scan(state, metaDataStateStore); - // MergeState newState = stats.nextMergeState(accumuloClient, state); - // assertEquals(MergeState.WAITING_FOR_OFFLINE, newState); - - // unassign the tablets - try (BatchDeleter deleter = - accumuloClient.createBatchDeleter(MetadataTable.NAME, Authorizations.EMPTY, 1000)) { - deleter.fetchColumnFamily(CurrentLocationColumnFamily.NAME); - deleter.setRanges(Collections.singletonList(new Range())); - deleter.delete(); - } - - // now we should be ready to merge but, we have inconsistent metadata - stats = scan(state, metaDataStateStore); - assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(accumuloClient, state)); - - // finish the split - KeyExtent tablet = new KeyExtent(tableId, new Text("p"), new Text("o")); - m = TabletColumnFamily.createPrevRowMutation(tablet); - TabletColumnFamily.SPLIT_RATIO_COLUMN.put(m, new Value("0.5")); - update(accumuloClient, m); - metaDataStateStore - .setLocations(Collections.singletonList(new Assignment(tablet, state.someTServer, null))); - - stats = scan(state, metaDataStateStore); - assertEquals(MergeState.WAITING_FOR_OFFLINE, stats.nextMergeState(accumuloClient, state)); - - // take it offline - m = TabletColumnFamily.createPrevRowMutation(tablet); - List walogs = Collections.emptyList(); - metaDataStateStore.unassign( - Collections.singletonList(TabletMetadata.builder(tablet) - .putLocation(Location.current(state.someTServer)) - .putHostingGoal(TabletHostingGoal.ALWAYS).build(ColumnType.LAST, ColumnType.SUSPEND)), - null); - - // now we can split - stats = scan(state, metaDataStateStore); - assertEquals(MergeState.MERGING, stats.nextMergeState(accumuloClient, state)); - } - } - - private MergeStats scan(MockCurrentState state, TabletStateStore metaDataStateStore) { - MergeStats stats = new MergeStats(state.mergeInfo); - stats.getMergeInfo().setState(MergeState.WAITING_FOR_OFFLINE); - for (TabletManagement tm : metaDataStateStore) { - TabletMetadata tabletMetadata = tm.getTabletMetadata(); - stats.update(tm.getTabletMetadata().getExtent(), - TabletState.compute(tabletMetadata, state.onlineTabletServers())); - } - return stats; - } -} diff --git a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java index 49f785a9ab5..cd183ef364c 100644 --- a/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java +++ b/test/src/main/java/org/apache/accumulo/test/manager/SuspendedTabletsIT.java @@ -72,6 +72,7 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +81,7 @@ import com.google.common.collect.SetMultimap; import com.google.common.net.HostAndPort; +@Disabled // ELASTICITY_TODO public class SuspendedTabletsIT extends ConfigurableMacBase { private static final Logger log = LoggerFactory.getLogger(SuspendedTabletsIT.class); private static ExecutorService THREAD_POOL; diff --git a/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java b/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java index f84d5138b82..9502dca9d96 100644 --- a/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java +++ b/test/src/main/java/org/apache/accumulo/test/util/FileMetadataUtil.java @@ -23,17 +23,25 @@ import java.util.HashMap; import java.util.Map; import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeMap; +import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.TableId; +import org.apache.accumulo.core.dataImpl.KeyExtent; import org.apache.accumulo.core.metadata.StoredTabletFile; +import org.apache.accumulo.core.metadata.schema.Ample.TabletMutator; import org.apache.accumulo.core.metadata.schema.DataFileValue; import org.apache.accumulo.core.metadata.schema.TabletMetadata; import org.apache.accumulo.core.metadata.schema.TabletMetadata.ColumnType; import org.apache.accumulo.core.metadata.schema.TabletsMetadata; import org.apache.accumulo.server.ServerContext; +import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.base.Preconditions; + public class FileMetadataUtil { private static final Logger log = LoggerFactory.getLogger(FileMetadataUtil.class); @@ -53,8 +61,8 @@ public class FileMetadataUtil { // Read each file referenced by that table int i = 0; for (TabletMetadata tabletMetadata : tabletsMetadata) { - for (Entry fileEntry : tabletMetadata.getFilesMap() - .entrySet()) { + for (Entry fileEntry : new TreeMap<>(tabletMetadata.getFilesMap()).entrySet()) { StoredTabletFile file = fileEntry.getKey(); DataFileValue dfv = fileEntry.getValue(); files.put(file, dfv); @@ -73,4 +81,81 @@ public class FileMetadataUtil { return files; } + + public static int countFiles(final ServerContext ctx, String tableName) { + return countFiles(ctx, tableName, null, null); + } + + public static int countFiles(final ServerContext ctx, String tableName, Text tabletStartRow, + Text tabletEndRow) { + final TableId tableId = TableId.of(ctx.tableOperations().tableIdMap().get(tableName)); + try (TabletsMetadata tabletsMetadata = ctx.getAmple().readTablets().forTable(tableId) + .overlapping(tabletStartRow, tabletEndRow).fetch(ColumnType.FILES).build()) { + return tabletsMetadata.stream().mapToInt(tm -> tm.getFilesMap().size()).sum(); + } + } + + public static void splitFilesIntoRanges(final ServerContext ctx, String tableName, + Set fileRanges) throws Exception { + splitFilesIntoRanges(ctx, tableName, null, null, fileRanges); + } + + public static void splitFilesIntoRanges(final ServerContext ctx, String tableName, + Text tabletStartRow, Text tabletEndRow, Set fileRanges) throws Exception { + + Preconditions.checkArgument(!fileRanges.isEmpty(), "Ranges must not be empty"); + + mutateTabletFiles(ctx, tableName, tabletStartRow, tabletEndRow, (tm, mutator, file, value) -> { + // Create a mutation to delete the existing file metadata entry with infinite range + mutator.deleteFile(file); + + fileRanges.forEach(range -> { + final DataFileValue newValue = + new DataFileValue(Integer.max(1, (int) (value.getSize() / fileRanges.size())), + Integer.max(1, (int) (value.getNumEntries() / fileRanges.size()))); + mutator.putFile(StoredTabletFile.of(file.getPath(), range), newValue); + }); + }); + } + + public static void mutateTabletFiles(final ServerContext ctx, String tableName, + Text tabletStartRow, Text tabletEndRow, FileMutator fileMutator) throws Exception { + + final TableId tableId = TableId.of(ctx.tableOperations().tableIdMap().get(tableName)); + + // Bring tablet offline so we can modify file metadata + ctx.tableOperations().offline(tableName, true); + + try (TabletsMetadata tabletsMetadata = + ctx.getAmple().readTablets().forTable(tableId).overlapping(tabletStartRow, tabletEndRow) + .fetch(ColumnType.FILES, ColumnType.PREV_ROW).build()) { + + // Process each tablet in the given start/end row range + for (TabletMetadata tabletMetadata : tabletsMetadata) { + final KeyExtent ke = tabletMetadata.getExtent(); + + TabletMutator mutator = ctx.getAmple().mutateTablet(ke); + + // Read each file and mutate + for (Entry fileEntry : tabletMetadata.getFilesMap() + .entrySet()) { + StoredTabletFile file = fileEntry.getKey(); + DataFileValue value = fileEntry.getValue(); + + // do any file mutations + fileMutator.mutate(tabletMetadata, mutator, file, value); + + mutator.mutate(); + } + } + } + + // Bring back online after metadata updates + ctx.tableOperations().online(tableName, true); + } + + public interface FileMutator { + void mutate(TabletMetadata tm, TabletMutator mutator, StoredTabletFile file, + DataFileValue value); + } }