Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
Expand Down Expand Up @@ -237,10 +236,7 @@ private void getFromTranslog(
final var retryingListener = listener.delegateResponse((l, e) -> {
final var cause = ExceptionsHelper.unwrapCause(e);
logger.debug("get_from_translog failed", cause);
if (cause instanceof ShardNotFoundException
|| cause instanceof IndexNotFoundException
|| cause instanceof AlreadyClosedException) {
// TODO AlreadyClosedException the engine reset should be fixed by ES-10826
if (cause instanceof ShardNotFoundException || cause instanceof IndexNotFoundException) {
logger.debug("retrying get_from_translog");
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
Expand All @@ -255,13 +251,7 @@ public void onClusterServiceClose() {

@Override
public void onTimeout(TimeValue timeout) {
// TODO AlreadyClosedException the engine reset should be fixed by ES-10826
if (cause instanceof AlreadyClosedException) {
// Do an additional retry just in case AlreadyClosedException didn't generate a cluster update
tryGetFromTranslog(request, indexShard, node, l);
} else {
l.onFailure(new ElasticsearchException("Timed out retrying get_from_translog", cause));
}
l.onFailure(new ElasticsearchException("Timed out retrying get_from_translog", cause));
}
});
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,15 +16,16 @@
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.core.Nullable;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.InternalEngine;
import org.elasticsearch.index.get.GetResult;
Expand All @@ -49,42 +50,47 @@ public class TransportGetFromTranslogAction extends HandledTransportAction<
public static final Logger logger = LogManager.getLogger(TransportGetFromTranslogAction.class);

private final IndicesService indicesService;
private final ThreadPool threadPool;

@Inject
public TransportGetFromTranslogAction(TransportService transportService, IndicesService indicesService, ActionFilters actionFilters) {
super(NAME, transportService, actionFilters, Request::new, transportService.getThreadPool().executor(ThreadPool.Names.GET));
this.indicesService = indicesService;
this.threadPool = transportService.getThreadPool();
}

@Override
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
final GetRequest getRequest = request.getRequest();
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
assert indexShard.routingEntry().isPromotableToPrimary() : "not an indexing shard" + indexShard.routingEntry();
assert getRequest.realtime();
ActionListener.completeWith(listener, () -> {
var result = indexShard.getService()
.getFromTranslog(
getRequest.id(),
getRequest.storedFields(),
getRequest.realtime(),
getRequest.version(),
getRequest.versionType(),
getRequest.fetchSourceContext(),
getRequest.isForceSyntheticSource()
);
long segmentGeneration = -1;
if (result == null) {
Engine engine = indexShard.getEngineOrNull();
if (engine == null) {
throw new AlreadyClosedException("engine closed");

SubscribableListener.<IndexShard>newForked(l -> {
var indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
assert indexShard.routingEntry().isPromotableToPrimary() : "not an indexing shard" + indexShard.routingEntry();
indexShard.ensureMutable(l.map(unused -> indexShard));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do real-time (m)GETs need to ensureMutable? I do not think we need a full unhollow engine for them. They should be able to work on hollow shards -- always returning null so that the search shard can service them themselves.

}).<Response>andThen((l, indexShard) -> {
threadPool.executor(ThreadPool.Names.GET).execute(ActionRunnable.supply(l, () -> {
var result = indexShard.getService()
.getFromTranslog(
getRequest.id(),
getRequest.storedFields(),
getRequest.realtime(),
getRequest.version(),
getRequest.versionType(),
getRequest.fetchSourceContext(),
getRequest.isForceSyntheticSource()
);
long segmentGeneration = -1;
if (result == null) {
Engine engine = indexShard.getEngineOrNull();
if (engine == null) {
throw new AlreadyClosedException("engine closed");
}
segmentGeneration = engine.getLastUnsafeSegmentGenerationForGets();
}
segmentGeneration = engine.getLastUnsafeSegmentGenerationForGets();
}
return new Response(result, indexShard.getOperationPrimaryTerm(), segmentGeneration);
});
return new Response(result, indexShard.getOperationPrimaryTerm(), segmentGeneration);
}));
}).addListener(listener);
}

public static class Request extends ActionRequest implements IndicesRequest {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@

import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
Expand Down Expand Up @@ -218,10 +217,7 @@ private void shardMultiGetFromTranslog(
final var retryingListener = listener.delegateResponse((l, e) -> {
final var cause = ExceptionsHelper.unwrapCause(e);
logger.debug("mget_from_translog[shard] failed", cause);
if (cause instanceof ShardNotFoundException
|| cause instanceof IndexNotFoundException
|| cause instanceof AlreadyClosedException) {
// TODO AlreadyClosedException the engine reset should be fixed by ES-10826
if (cause instanceof ShardNotFoundException || cause instanceof IndexNotFoundException) {
logger.debug("retrying mget_from_translog[shard]");
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
Expand All @@ -236,13 +232,7 @@ public void onClusterServiceClose() {

@Override
public void onTimeout(TimeValue timeout) {
// TODO AlreadyClosedException the engine reset should be fixed by ES-10826
if (cause instanceof AlreadyClosedException) {
// Do an additional retry just in case AlreadyClosedException didn't generate a cluster update
tryShardMultiGetFromTranslog(request, indexShard, node, l);
} else {
l.onFailure(new ElasticsearchException("Timed out retrying mget_from_translog[shard]", cause));
}
l.onFailure(new ElasticsearchException("Timed out retrying mget_from_translog[shard]", cause));
}
});
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,14 @@
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.SubscribableListener;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
Expand All @@ -43,6 +44,7 @@ public class TransportShardMultiGetFomTranslogAction extends HandledTransportAct
public static final Logger logger = LogManager.getLogger(TransportShardMultiGetFomTranslogAction.class);

private final IndicesService indicesService;
private final ThreadPool threadPool;

protected TransportShardMultiGetFomTranslogAction(
TransportService transportService,
Expand All @@ -51,60 +53,64 @@ protected TransportShardMultiGetFomTranslogAction(
) {
super(NAME, transportService, actionFilters, Request::new, transportService.getThreadPool().executor(ThreadPool.Names.GET));
this.indicesService = indicesService;
this.threadPool = transportService.getThreadPool();
}

@Override
protected void doExecute(Task task, Request request, ActionListener<Response> listener) {
var multiGetShardRequest = request.getMultiGetShardRequest();
var shardId = request.getShardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
assert indexShard.routingEntry().isPromotableToPrimary() : "not an indexing shard" + indexShard.routingEntry();
assert multiGetShardRequest.realtime();
ActionListener.completeWith(listener, () -> {
var multiGetShardResponse = new MultiGetShardResponse();
var someItemsNotFoundInTranslog = false;
for (int i = 0; i < multiGetShardRequest.locations.size(); i++) {
var item = multiGetShardRequest.items.get(i);
try {
var result = indexShard.getService()
.getFromTranslog(
item.id(),
item.storedFields(),
multiGetShardRequest.realtime(),
item.version(),
item.versionType(),
item.fetchSourceContext(),
multiGetShardRequest.isForceSyntheticSource()

SubscribableListener.<IndexShard>newForked(l -> {
var indexShard = indicesService.indexServiceSafe(request.getShardId().getIndex()).getShard(request.getShardId().id());
assert indexShard.routingEntry().isPromotableToPrimary() : "not an indexing shard" + indexShard.routingEntry();
indexShard.ensureMutable(l.map(unused -> indexShard));
}).<Response>andThen((l, indexShard) -> {
threadPool.executor(ThreadPool.Names.GET).execute(ActionRunnable.supply(l, () -> {
var multiGetShardResponse = new MultiGetShardResponse();
var someItemsNotFoundInTranslog = false;
for (int i = 0; i < multiGetShardRequest.locations.size(); i++) {
var item = multiGetShardRequest.items.get(i);
try {
var result = indexShard.getService()
.getFromTranslog(
item.id(),
item.storedFields(),
multiGetShardRequest.realtime(),
item.version(),
item.versionType(),
item.fetchSourceContext(),
multiGetShardRequest.isForceSyntheticSource()
);
GetResponse getResponse = null;
if (result == null) {
someItemsNotFoundInTranslog = true;
} else {
getResponse = new GetResponse(result);
}
multiGetShardResponse.add(multiGetShardRequest.locations.get(i), getResponse);
} catch (RuntimeException | IOException e) {
if (TransportActions.isShardNotAvailableException(e)) {
throw e;
}
logger.debug("failed to execute multi_get_from_translog for {}[id={}]: {}", request.getShardId(), item.id(), e);
multiGetShardResponse.add(
multiGetShardRequest.locations.get(i),
new MultiGetResponse.Failure(multiGetShardRequest.index(), item.id(), e)
);
GetResponse getResponse = null;
if (result == null) {
someItemsNotFoundInTranslog = true;
} else {
getResponse = new GetResponse(result);
}
multiGetShardResponse.add(multiGetShardRequest.locations.get(i), getResponse);
} catch (RuntimeException | IOException e) {
if (TransportActions.isShardNotAvailableException(e)) {
throw e;
}
logger.debug("failed to execute multi_get_from_translog for {}[id={}]: {}", shardId, item.id(), e);
multiGetShardResponse.add(
multiGetShardRequest.locations.get(i),
new MultiGetResponse.Failure(multiGetShardRequest.index(), item.id(), e)
);
}
}
long segmentGeneration = -1;
if (someItemsNotFoundInTranslog) {
Engine engine = indexShard.getEngineOrNull();
if (engine == null) {
throw new AlreadyClosedException("engine closed");
long segmentGeneration = -1;
if (someItemsNotFoundInTranslog) {
Engine engine = indexShard.getEngineOrNull();
if (engine == null) {
throw new AlreadyClosedException("engine closed");
}
segmentGeneration = engine.getLastUnsafeSegmentGenerationForGets();
}
segmentGeneration = engine.getLastUnsafeSegmentGenerationForGets();
}
return new Response(multiGetShardResponse, indexShard.getOperationPrimaryTerm(), segmentGeneration);
});
return new Response(multiGetShardResponse, indexShard.getOperationPrimaryTerm(), segmentGeneration);
}));
}).addListener(listener);
}

public static class Request extends ActionRequest {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the "Elastic License
* 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side
* Public License v 1"; you may not use this file except in compliance with, at
* your election, the "Elastic License 2.0", the "GNU Affero General Public
* License v3.0 only", or the "Server Side Public License, v 1".
*/

package org.elasticsearch.index.engine;

import org.apache.lucene.search.ReferenceManager;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;

import java.io.IOException;
import java.util.List;
import java.util.Objects;

public class ElasticsearchDirectoryReaderRefreshListener {

private final ReferenceManager<ElasticsearchDirectoryReader> readerManager;
private final List<ReferenceManager.RefreshListener> listeners;

public ElasticsearchDirectoryReaderRefreshListener(
ReferenceManager<ElasticsearchDirectoryReader> readerManager,
List<ReferenceManager.RefreshListener> listeners
) {
this.readerManager = Objects.requireNonNull(readerManager);
this.listeners = List.copyOf(listeners);
this.readerManager.addListener(new InternalRefreshListener());
}

private class InternalRefreshListener implements ReferenceManager.RefreshListener {

@Override
public void beforeRefresh() throws IOException {
for (var listener : listeners) {
listener.beforeRefresh();
}
}

@Override
public void afterRefresh(boolean didRefresh) throws IOException {
var reader = readerManager.acquire();
try {
for (var listener : listeners) {
if (listener instanceof ReaderAwareRefreshListener l) {
l.afterRefresh(didRefresh, reader);
} else {
listener.afterRefresh(didRefresh);
}
}
} finally {
readerManager.release(reader);
}
}
}
}
12 changes: 10 additions & 2 deletions server/src/main/java/org/elasticsearch/index/engine/Engine.java
Original file line number Diff line number Diff line change
Expand Up @@ -989,6 +989,10 @@ protected final void ensureOpen() {
ensureOpen(null);
}

public boolean isMutable() {
return true;
}

/** get commits stats for the last commit */
public final CommitStats commitStats() {
return new CommitStats(getLastCommittedSegmentInfos());
Expand Down Expand Up @@ -2341,13 +2345,17 @@ public record FlushResult(boolean flushPerformed, long generation) {
}

/**
* Ensures the engine is in a state that it can be closed by a call to {@link IndexShard#resetEngine()}.
* Ensures the engine is in a state that it can be closed by a call to {@link IndexShard#resetEngine(boolean)}.
*
* In general, resetting the engine should be done with care, to consider any
* in-progress operations and listeners (e.g., primary term and generation listeners).
* At the moment, this is implemented in serverless for a special case that ensures the engine is prepared for reset.
*/
public void prepareForEngineReset() throws IOException {
public void beforeReset() throws IOException {
throw new UnsupportedOperationException("does not support engine reset");
}

public void afterReset() throws IOException {
throw new UnsupportedOperationException("does not support engine reset");
}

Expand Down
Loading