Merge pull request #15235 from s1monw/remove_recovery_settings

Restore chunksize of 512kb on recovery and remove configurability
This commit is contained in:
Simon Willnauer 2015-12-11 15:27:20 +01:00
commit 90ff1ad5fb
10 changed files with 66 additions and 159 deletions

View File

@ -164,10 +164,6 @@ public class ClusterModule extends AbstractModule {
registerClusterDynamicSetting(IndicesTTLService.INDICES_TTL_INTERVAL, Validator.TIME);
registerClusterDynamicSetting(MappingUpdatedAction.INDICES_MAPPING_DYNAMIC_TIMEOUT, Validator.TIME);
registerClusterDynamicSetting(MetaData.SETTING_READ_ONLY, Validator.EMPTY);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, Validator.POSITIVE_BYTES_SIZE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_OPS, Validator.INTEGER);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, Validator.BYTES_SIZE);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_COMPRESS, Validator.EMPTY);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, Validator.POSITIVE_INTEGER);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, Validator.POSITIVE_INTEGER);
registerClusterDynamicSetting(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, Validator.BYTES_SIZE);

View File

@ -746,8 +746,6 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, Fr
/** All known byte-sized cluster settings. */
public static final Set<String> CLUSTER_BYTES_SIZE_SETTINGS = unmodifiableSet(newHashSet(
IndexStoreConfig.INDICES_STORE_THROTTLE_MAX_BYTES_PER_SEC,
RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE,
RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE,
RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC));

View File

@ -20,6 +20,8 @@
package org.elasticsearch.common.io;
import java.nio.charset.StandardCharsets;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.util.Callback;
import java.io.BufferedReader;
@ -68,6 +70,7 @@ public abstract class Streams {
public static long copy(InputStream in, OutputStream out, byte[] buffer) throws IOException {
Objects.requireNonNull(in, "No InputStream specified");
Objects.requireNonNull(out, "No OutputStream specified");
boolean success = false;
try {
long byteCount = 0;
int bytesRead;
@ -76,17 +79,13 @@ public abstract class Streams {
byteCount += bytesRead;
}
out.flush();
success = true;
return byteCount;
} finally {
try {
in.close();
} catch (IOException ex) {
// do nothing
}
try {
out.close();
} catch (IOException ex) {
// do nothing
if (success) {
IOUtils.close(in, out);
} else {
IOUtils.closeWhileHandlingException(in, out);
}
}
}
@ -130,6 +129,7 @@ public abstract class Streams {
public static int copy(Reader in, Writer out) throws IOException {
Objects.requireNonNull(in, "No Reader specified");
Objects.requireNonNull(out, "No Writer specified");
boolean success = false;
try {
int byteCount = 0;
char[] buffer = new char[BUFFER_SIZE];
@ -139,17 +139,13 @@ public abstract class Streams {
byteCount += bytesRead;
}
out.flush();
success = true;
return byteCount;
} finally {
try {
in.close();
} catch (IOException ex) {
// do nothing
}
try {
out.close();
} catch (IOException ex) {
// do nothing
if (success) {
IOUtils.close(in, out);
} else {
IOUtils.closeWhileHandlingException(in, out);
}
}
}

View File

@ -40,10 +40,6 @@ import java.util.concurrent.TimeUnit;
*/
public class RecoverySettings extends AbstractComponent implements Closeable {
public static final String INDICES_RECOVERY_FILE_CHUNK_SIZE = "indices.recovery.file_chunk_size";
public static final String INDICES_RECOVERY_TRANSLOG_OPS = "indices.recovery.translog_ops";
public static final String INDICES_RECOVERY_TRANSLOG_SIZE = "indices.recovery.translog_size";
public static final String INDICES_RECOVERY_COMPRESS = "indices.recovery.compress";
public static final String INDICES_RECOVERY_CONCURRENT_STREAMS = "indices.recovery.concurrent_streams";
public static final String INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS = "indices.recovery.concurrent_small_file_streams";
public static final String INDICES_RECOVERY_MAX_BYTES_PER_SEC = "indices.recovery.max_bytes_per_sec";
@ -75,11 +71,7 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
public static final long SMALL_FILE_CUTOFF_BYTES = ByteSizeValue.parseBytesSizeValue("5mb", "SMALL_FILE_CUTOFF_BYTES").bytes();
private volatile ByteSizeValue fileChunkSize;
private volatile boolean compress;
private volatile int translogOps;
private volatile ByteSizeValue translogSize;
public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue(512, ByteSizeUnit.KB);
private volatile int concurrentStreams;
private volatile int concurrentSmallFileStreams;
@ -94,16 +86,12 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
private volatile TimeValue internalActionTimeout;
private volatile TimeValue internalActionLongTimeout;
private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE;
@Inject
public RecoverySettings(Settings settings, NodeSettingsService nodeSettingsService) {
super(settings);
this.fileChunkSize = settings.getAsBytesSize(INDICES_RECOVERY_FILE_CHUNK_SIZE, new ByteSizeValue(512, ByteSizeUnit.KB));
this.translogOps = settings.getAsInt(INDICES_RECOVERY_TRANSLOG_OPS, 1000);
this.translogSize = settings.getAsBytesSize(INDICES_RECOVERY_TRANSLOG_SIZE, new ByteSizeValue(512, ByteSizeUnit.KB));
this.compress = settings.getAsBoolean(INDICES_RECOVERY_COMPRESS, true);
this.retryDelayStateSync = settings.getAsTime(INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC, TimeValue.timeValueMillis(500));
// doesn't have to be fast as nodes are reconnected every 10s by default (see InternalClusterService.ReconnectToNodes)
// and we want to give the master time to remove a faulty node
@ -132,8 +120,8 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.mbFrac());
}
logger.debug("using max_bytes_per_sec[{}], concurrent_streams [{}], file_chunk_size [{}], translog_size [{}], translog_ops [{}], and compress [{}]",
maxBytesPerSec, concurrentStreams, fileChunkSize, translogSize, translogOps, compress);
logger.debug("using max_bytes_per_sec[{}], concurrent_streams [{}]",
maxBytesPerSec, concurrentStreams);
nodeSettingsService.addListener(new ApplySettings());
}
@ -144,26 +132,6 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
ThreadPool.terminate(concurrentSmallFileStreamPool, 1, TimeUnit.SECONDS);
}
public ByteSizeValue fileChunkSize() {
return fileChunkSize;
}
public boolean compress() {
return compress;
}
public int translogOps() {
return translogOps;
}
public ByteSizeValue translogSize() {
return translogSize;
}
public int concurrentStreams() {
return concurrentStreams;
}
public ThreadPoolExecutor concurrentStreamPool() {
return concurrentStreamPool;
}
@ -196,6 +164,15 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
return internalActionLongTimeout;
}
public ByteSizeValue getChunkSize() { return chunkSize; }
void setChunkSize(ByteSizeValue chunkSize) { // only settable for tests
if (chunkSize.bytesAsInt() <= 0) {
throw new IllegalArgumentException("chunkSize must be > 0");
}
this.chunkSize = chunkSize;
}
class ApplySettings implements NodeSettingsService.Listener {
@Override
@ -213,30 +190,6 @@ public class RecoverySettings extends AbstractComponent implements Closeable {
}
}
ByteSizeValue fileChunkSize = settings.getAsBytesSize(INDICES_RECOVERY_FILE_CHUNK_SIZE, RecoverySettings.this.fileChunkSize);
if (!fileChunkSize.equals(RecoverySettings.this.fileChunkSize)) {
logger.info("updating [indices.recovery.file_chunk_size] from [{}] to [{}]", RecoverySettings.this.fileChunkSize, fileChunkSize);
RecoverySettings.this.fileChunkSize = fileChunkSize;
}
int translogOps = settings.getAsInt(INDICES_RECOVERY_TRANSLOG_OPS, RecoverySettings.this.translogOps);
if (translogOps != RecoverySettings.this.translogOps) {
logger.info("updating [indices.recovery.translog_ops] from [{}] to [{}]", RecoverySettings.this.translogOps, translogOps);
RecoverySettings.this.translogOps = translogOps;
}
ByteSizeValue translogSize = settings.getAsBytesSize(INDICES_RECOVERY_TRANSLOG_SIZE, RecoverySettings.this.translogSize);
if (!translogSize.equals(RecoverySettings.this.translogSize)) {
logger.info("updating [indices.recovery.translog_size] from [{}] to [{}]", RecoverySettings.this.translogSize, translogSize);
RecoverySettings.this.translogSize = translogSize;
}
boolean compress = settings.getAsBoolean(INDICES_RECOVERY_COMPRESS, RecoverySettings.this.compress);
if (compress != RecoverySettings.this.compress) {
logger.info("updating [indices.recovery.compress] from [{}] to [{}]", RecoverySettings.this.compress, compress);
RecoverySettings.this.compress = compress;
}
int concurrentStreams = settings.getAsInt(INDICES_RECOVERY_CONCURRENT_STREAMS, RecoverySettings.this.concurrentStreams);
if (concurrentStreams != RecoverySettings.this.concurrentStreams) {
logger.info("updating [indices.recovery.concurrent_streams] from [{}] to [{}]", RecoverySettings.this.concurrentStreams, concurrentStreams);

View File

@ -36,7 +36,9 @@ import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CancellableThreads;
import org.elasticsearch.common.util.CancellableThreads.Interruptable;
import org.elasticsearch.index.engine.RecoveryEngineException;
@ -49,6 +51,7 @@ import org.elasticsearch.transport.RemoteTransportException;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportService;
import java.io.BufferedOutputStream;
import java.io.IOException;
import java.io.OutputStream;
import java.util.ArrayList;
@ -57,6 +60,7 @@ import java.util.List;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Function;
import java.util.stream.StreamSupport;
@ -77,9 +81,9 @@ public class RecoverySourceHandler {
private final StartRecoveryRequest request;
private final RecoverySettings recoverySettings;
private final TransportService transportService;
private final int chunkSizeInBytes;
protected final RecoveryResponse response;
private final TransportRequestOptions requestOptions;
private final CancellableThreads cancellableThreads = new CancellableThreads() {
@Override
@ -106,14 +110,8 @@ public class RecoverySourceHandler {
this.transportService = transportService;
this.indexName = this.request.shardId().index().name();
this.shardId = this.request.shardId().id();
this.chunkSizeInBytes = recoverySettings.getChunkSize().bytesAsInt();
this.response = new RecoveryResponse();
this.requestOptions = TransportRequestOptions.builder()
.withCompress(recoverySettings.compress())
.withType(TransportRequestOptions.Type.RECOVERY)
.withTimeout(recoverySettings.internalActionTimeout())
.build();
}
/**
@ -249,7 +247,7 @@ public class RecoverySourceHandler {
});
// How many bytes we've copied since we last called RateLimiter.pause
final AtomicLong bytesSinceLastPause = new AtomicLong();
final Function<StoreFileMetaData, OutputStream> outputStreamFactories = (md) -> new RecoveryOutputStream(md, bytesSinceLastPause, translogView);
final Function<StoreFileMetaData, OutputStream> outputStreamFactories = (md) -> new BufferedOutputStream(new RecoveryOutputStream(md, bytesSinceLastPause, translogView), chunkSizeInBytes);
sendFiles(store, phase1Files.toArray(new StoreFileMetaData[phase1Files.size()]), outputStreamFactories);
cancellableThreads.execute(() -> {
// Send the CLEAN_FILES request, which takes all of the files that
@ -432,7 +430,7 @@ public class RecoverySourceHandler {
}
final TransportRequestOptions recoveryOptions = TransportRequestOptions.builder()
.withCompress(recoverySettings.compress())
.withCompress(true)
.withType(TransportRequestOptions.Type.RECOVERY)
.withTimeout(recoverySettings.internalActionLongTimeout())
.build();
@ -451,9 +449,9 @@ public class RecoverySourceHandler {
size += operation.estimateSize();
totalOperations++;
// Check if this request is past the size or bytes threshold, and
// Check if this request is past bytes threshold, and
// if so, send it off
if (ops >= recoverySettings.translogOps() || size >= recoverySettings.translogSize().bytes()) {
if (size >= chunkSizeInBytes) {
// don't throttle translog, since we lock for phase3 indexing,
// so we need to move it as fast as possible. Note, since we
@ -537,7 +535,7 @@ public class RecoverySourceHandler {
@Override
public final void write(int b) throws IOException {
write(new byte[]{(byte) b}, 0, 1);
throw new UnsupportedOperationException("we can't send single bytes over the wire");
}
@Override
@ -548,6 +546,11 @@ public class RecoverySourceHandler {
}
private void sendNextChunk(long position, BytesArray content, boolean lastChunk) throws IOException {
final TransportRequestOptions chunkSendOptions = TransportRequestOptions.builder()
.withCompress(false) // lucene files are already compressed and therefore compressing this won't really help much so we are safing the cpu for other things
.withType(TransportRequestOptions.Type.RECOVERY)
.withTimeout(recoverySettings.internalActionTimeout())
.build();
cancellableThreads.execute(() -> {
// Pause using the rate limiter, if desired, to throttle the recovery
final long throttleTimeInNanos;
@ -577,7 +580,7 @@ public class RecoverySourceHandler {
* see how many translog ops we accumulate while copying files across the network. A future optimization
* would be in to restart file copy again (new deltas) if we have too many translog ops are piling up.
*/
throttleTimeInNanos), requestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
throttleTimeInNanos), chunkSendOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
});
if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us
throw new IndexShardClosedException(request.shardId());
@ -670,9 +673,10 @@ public class RecoverySourceHandler {
pool = recoverySettings.concurrentSmallFileStreamPool();
}
Future<Void> future = pool.submit(() -> {
try (final OutputStream outputStream = outputStreamFactory.apply(md);
final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE)) {
Streams.copy(new InputStreamIndexInput(indexInput, md.length()), outputStream);
try (final IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE)) {
// it's fine that we are only having the indexInput int he try/with block. The copy methods handles
// exceptions during close correctly and doesn't hide the original exception.
Streams.copy(new InputStreamIndexInput(indexInput, md.length()), outputStreamFactory.apply(md));
}
return null;
});

View File

@ -138,21 +138,25 @@ public class IndexRecoveryIT extends ESIntegTestCase {
}
private void slowDownRecovery(ByteSizeValue shardSize) {
long chunkSize = shardSize.bytes() / 10;
long chunkSize = Math.max(1, shardSize.bytes() / 10);
for(RecoverySettings settings : internalCluster().getInstances(RecoverySettings.class)) {
setChunkSize(settings, new ByteSizeValue(chunkSize, ByteSizeUnit.BYTES));
}
assertTrue(client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder()
// one chunk per sec..
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, chunkSize, ByteSizeUnit.BYTES)
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, chunkSize, ByteSizeUnit.BYTES)
)
.get().isAcknowledged());
}
private void restoreRecoverySpeed() {
for(RecoverySettings settings : internalCluster().getInstances(RecoverySettings.class)) {
setChunkSize(settings, RecoverySettings.DEFAULT_CHUNK_SIZE);
}
assertTrue(client().admin().cluster().prepareUpdateSettings()
.setTransientSettings(Settings.builder()
.put(RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC, "20mb")
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, "512kb")
)
.get().isAcknowledged());
}
@ -631,4 +635,8 @@ public class IndexRecoveryIT extends ESIntegTestCase {
transport.sendRequest(node, requestId, action, request, options);
}
}
public static void setChunkSize(RecoverySettings recoverySettings, ByteSizeValue chunksSize) {
recoverySettings.setChunkSize(chunksSize);
}
}

View File

@ -32,24 +32,6 @@ public class RecoverySettingsTests extends ESSingleNodeTestCase {
}
public void testAllSettingsAreDynamicallyUpdatable() {
innerTestSettings(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
@Override
public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.fileChunkSize().bytesAsInt());
}
});
innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_OPS, randomIntBetween(1, 200), new Validator() {
@Override
public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.translogOps());
}
});
innerTestSettings(RecoverySettings.INDICES_RECOVERY_TRANSLOG_SIZE, randomIntBetween(1, 200), ByteSizeUnit.BYTES, new Validator() {
@Override
public void validate(RecoverySettings recoverySettings, int expectedValue) {
assertEquals(expectedValue, recoverySettings.translogSize().bytesAsInt());
}
});
innerTestSettings(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, randomIntBetween(1, 200), new Validator() {
@Override
public void validate(RecoverySettings recoverySettings, int expectedValue) {
@ -98,13 +80,6 @@ public class RecoverySettingsTests extends ESSingleNodeTestCase {
assertEquals(expectedValue, recoverySettings.internalActionLongTimeout().millis());
}
});
innerTestSettings(RecoverySettings.INDICES_RECOVERY_COMPRESS, false, new Validator() {
@Override
public void validate(RecoverySettings recoverySettings, boolean expectedValue) {
assertEquals(expectedValue, recoverySettings.compress());
}
});
}
private static class Validator {

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.recovery.IndexRecoveryIT;
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.recovery.RecoveryTarget;
@ -58,13 +59,6 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@ESIntegTestCase.ClusterScope(numDataNodes = 2, numClientNodes = 0, scope = ESIntegTestCase.Scope.TEST)
@SuppressCodecs("*") // test relies on exact file extensions
public class TruncatedRecoveryIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) {
Settings.Builder builder = Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES));
return builder.build();
}
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
@ -78,6 +72,10 @@ public class TruncatedRecoveryIT extends ESIntegTestCase {
* Later we allow full recovery to ensure we can still recover and don't run into corruptions.
*/
public void testCancelRecoveryAndResume() throws Exception {
for(RecoverySettings settings : internalCluster().getInstances(RecoverySettings.class)) {
IndexRecoveryIT.setChunkSize(settings, new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES));
}
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats().get();
List<NodeStats> dataNodeStats = new ArrayList<>();
for (NodeStats stat : nodeStats.getNodes()) {

View File

@ -131,16 +131,6 @@ public abstract class ESBackcompatTestCase extends ESIntegTestCase {
return file;
}
@Override
protected Settings.Builder setRandomIndexSettings(Random random, Settings.Builder builder) {
if (globalCompatibilityVersion().before(Version.V_1_3_2)) {
// if we test against nodes before 1.3.2 we disable all the compression due to a known bug
// see #7210
builder.put(RecoverySettings.INDICES_RECOVERY_COMPRESS, false);
}
return builder;
}
/**
* Retruns the tests compatibility version.
*/
@ -250,13 +240,6 @@ public abstract class ESBackcompatTestCase extends ESIntegTestCase {
Settings.Builder builder = Settings.builder().put(requiredSettings());
builder.put(TransportModule.TRANSPORT_TYPE_KEY, "netty"); // run same transport / disco as external
builder.put("node.mode", "network");
if (compatibilityVersion().before(Version.V_1_3_2)) {
// if we test against nodes before 1.3.2 we disable all the compression due to a known bug
// see #7210
builder.put(Transport.TransportSettings.TRANSPORT_TCP_COMPRESS, false)
.put(RecoverySettings.INDICES_RECOVERY_COMPRESS, false);
}
return builder.build();
}

View File

@ -447,10 +447,6 @@ public final class InternalTestCluster extends TestCluster {
}
}
if (random.nextBoolean()) {
builder.put(RecoverySettings.INDICES_RECOVERY_COMPRESS, random.nextBoolean());
}
if (random.nextBoolean()) {
builder.put(NettyTransport.PING_SCHEDULE, RandomInts.randomIntBetween(random, 100, 2000) + "ms");
}