Skip to content

Commit 89a10ac

Browse files
author
Himshikha Gupta
committed
Async write flow for routing table
1 parent 043a170 commit 89a10ac

File tree

6 files changed

+223
-111
lines changed

6 files changed

+223
-111
lines changed

server/src/main/java/org/opensearch/cluster/routing/remote/RemoteRoutingTableService.java

+122-65
Original file line numberDiff line numberDiff line change
@@ -10,41 +10,39 @@
1010

1111
import org.apache.logging.log4j.LogManager;
1212
import org.apache.logging.log4j.Logger;
13-
import org.opensearch.Version;
13+
import org.apache.lucene.store.IndexInput;
1414
import org.opensearch.action.LatchedActionListener;
1515
import org.opensearch.cluster.ClusterState;
1616
import org.opensearch.cluster.routing.IndexRoutingTable;
1717
import org.opensearch.cluster.routing.RoutingTable;
1818
import org.opensearch.common.CheckedRunnable;
19+
import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer;
1920
import org.opensearch.common.blobstore.BlobContainer;
2021
import org.opensearch.common.blobstore.BlobPath;
21-
import org.opensearch.common.io.stream.BytesStreamOutput;
22+
import org.opensearch.common.blobstore.stream.write.WritePriority;
23+
import org.opensearch.common.blobstore.transfer.RemoteTransferContainer;
24+
import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream;
2225

23-
import org.opensearch.common.blobstore.BlobContainer;
24-
import org.opensearch.common.settings.ClusterSettings;
26+
import org.opensearch.common.lucene.store.ByteArrayIndexInput;
2527
import org.opensearch.common.settings.Setting;
2628
import org.opensearch.common.settings.Settings;
2729
import org.opensearch.common.util.io.IOUtils;
2830
import org.opensearch.core.action.ActionListener;
29-
import org.opensearch.core.common.io.stream.StreamOutput;
3031
import org.opensearch.core.index.Index;
31-
import org.opensearch.core.xcontent.NamedXContentRegistry;
32-
import org.opensearch.core.xcontent.ToXContent;
3332
import org.opensearch.gateway.remote.ClusterMetadataManifest;
34-
import org.opensearch.gateway.remote.RemoteClusterStateService;
3533
import org.opensearch.gateway.remote.RemoteClusterStateUtils;
3634
import org.opensearch.gateway.remote.routingtable.IndexRoutingTableInputStream;
3735
import org.opensearch.gateway.remote.routingtable.IndexRoutingTableInputStreamReader;
36+
import org.opensearch.index.remote.RemoteStoreEnums;
37+
import org.opensearch.index.remote.RemoteStorePathStrategy;
3838
import org.opensearch.index.remote.RemoteStoreUtils;
3939
import org.opensearch.node.Node;
4040
import org.opensearch.node.remotestore.RemoteStoreNodeAttribute;
4141
import org.opensearch.repositories.RepositoriesService;
4242
import org.opensearch.repositories.Repository;
4343
import org.opensearch.repositories.blobstore.BlobStoreRepository;
44-
import org.opensearch.repositories.blobstore.ChecksumBlobStoreFormat;
4544
import org.opensearch.threadpool.ThreadPool;
4645

47-
import java.io.*;
4846
import java.io.Closeable;
4947
import java.io.IOException;
5048

@@ -53,11 +51,13 @@
5351
import java.util.HashMap;
5452
import java.util.List;
5553
import java.util.Map;
54+
import java.util.Set;
5655
import java.util.concurrent.ExecutorService;
5756
import java.util.function.Function;
5857
import java.util.function.Supplier;
5958
import java.util.stream.Collectors;
6059

60+
import static org.opensearch.common.blobstore.transfer.RemoteTransferContainer.checksumOfChecksum;
6161
import static org.opensearch.gateway.remote.RemoteClusterStateUtils.getCusterMetadataBasePath;
6262
import static org.opensearch.node.remotestore.RemoteStoreNodeAttribute.isRemoteRoutingTableEnabled;
6363

@@ -78,100 +78,157 @@ public class RemoteRoutingTableService implements Closeable {
7878
Setting.Property.Final
7979
);
8080
public static final String INDEX_ROUTING_PATH_TOKEN = "index-routing";
81+
public static final String INDEX_ROUTING_FILE_PREFIX = "index_routing";
8182
public static final String DELIMITER = "__";
82-
83+
public static final String INDEX_ROUTING_METADATA_PREFIX = "indexRouting--";
8384
private static final Logger logger = LogManager.getLogger(RemoteRoutingTableService.class);
8485
private final Settings settings;
8586
private final Supplier<RepositoriesService> repositoriesService;
86-
private final ClusterSettings clusterSettings;
8787
private BlobStoreRepository blobStoreRepository;
8888
private final ThreadPool threadPool;
8989

9090
public RemoteRoutingTableService(Supplier<RepositoriesService> repositoriesService,
9191
Settings settings,
92-
ClusterSettings clusterSettings, ThreadPool threadPool) {
92+
ThreadPool threadPool) {
9393
assert isRemoteRoutingTableEnabled(settings) : "Remote routing table is not enabled";
9494
this.repositoriesService = repositoriesService;
9595
this.settings = settings;
96-
this.clusterSettings = clusterSettings;
9796
this.threadPool = threadPool;
9897
}
9998

100-
public List<ClusterMetadataManifest.UploadedIndexMetadata> writeFullRoutingTable(ClusterState clusterState, String previousClusterUUID) {
99+
public List<IndexRoutingTable> getChangedIndicesRouting( ClusterState previousClusterState,
100+
ClusterState clusterState) {
101+
Map<String, IndexRoutingTable> previousIndexRoutingTable = previousClusterState.getRoutingTable().getIndicesRouting();
102+
List<IndexRoutingTable> changedIndicesRouting = new ArrayList<>();
103+
for (IndexRoutingTable indexRouting : clusterState.getRoutingTable().getIndicesRouting().values()) {
104+
if (!(previousIndexRoutingTable.containsKey(indexRouting.getIndex().getName()) && indexRouting.equals(previousIndexRoutingTable.get(indexRouting.getIndex().getName())))) {
105+
changedIndicesRouting.add(indexRouting);
106+
logger.info("changedIndicesRouting {}", indexRouting.prettyPrint());
107+
}
108+
}
101109

110+
return changedIndicesRouting;
111+
}
112+
113+
public CheckedRunnable<IOException> getIndexRoutingAsyncAction(
114+
ClusterState clusterState,
115+
IndexRoutingTable indexRouting,
116+
LatchedActionListener<ClusterMetadataManifest.UploadedMetadata> latchedActionListener
117+
) throws IOException {
102118

103-
//batch index count and parallelize
104-
RoutingTable currentRoutingTable = clusterState.getRoutingTable();
105-
List<ClusterMetadataManifest.UploadedIndexMetadata> uploadedIndices = new ArrayList<>();
106119
BlobPath custerMetadataBasePath = getCusterMetadataBasePath(blobStoreRepository, clusterState.getClusterName().value(),
107-
clusterState.metadata().clusterUUID());
108-
for (IndexRoutingTable indexRouting : currentRoutingTable.getIndicesRouting().values()) {
109-
uploadedIndices.add(uploadIndex(indexRouting, custerMetadataBasePath));
120+
clusterState.metadata().clusterUUID()).add(INDEX_ROUTING_PATH_TOKEN);
121+
logger.info("custerMetadataBasePath {}", custerMetadataBasePath);
122+
123+
BlobPath path = RemoteStoreEnums.PathType.HASHED_PREFIX.path(RemoteStorePathStrategy.PathInput.builder()
124+
.basePath(custerMetadataBasePath)
125+
.indexUUID(indexRouting.getIndex().getUUID())
126+
.build(),
127+
RemoteStoreEnums.PathHashAlgorithm.FNV_1A_BASE64);
128+
logger.info("path from prefix hasd {}", path);
129+
final BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer(path);
130+
131+
final String fileName = getIndexRoutingFileName();
132+
logger.info("fileName {}", fileName);
133+
134+
ActionListener<Void> completionListener = ActionListener.wrap(
135+
resp -> latchedActionListener.onResponse(
136+
new ClusterMetadataManifest.UploadedIndexMetadata(
137+
138+
indexRouting.getIndex().getName(),
139+
indexRouting.getIndex().getUUID(),
140+
path.buildAsString() + fileName,
141+
INDEX_ROUTING_METADATA_PREFIX
142+
)
143+
),
144+
ex -> latchedActionListener.onFailure(new RemoteClusterStateUtils.RemoteStateTransferException(indexRouting.getIndex().toString(), ex))
145+
);
146+
147+
if (blobContainer instanceof AsyncMultiStreamBlobContainer == false) {
148+
logger.info("TRYING FILE UPLOAD");
149+
150+
return () -> {
151+
logger.info("Going to upload {}", indexRouting.prettyPrint());
152+
153+
uploadIndex(indexRouting, fileName , blobContainer);
154+
logger.info("upload done {}", indexRouting.prettyPrint());
155+
156+
completionListener.onResponse(null);
157+
logger.info("response done {}", indexRouting.prettyPrint());
158+
159+
};
110160
}
111-
logger.info("uploadedIndices {}", uploadedIndices);
112161

113-
return uploadedIndices;
162+
logger.info("TRYING S3 UPLOAD");
163+
164+
//TODO: Integrate with S3AsyncCrtClient for using buffered stream directly with putObject.
165+
try (
166+
InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexRouting);
167+
IndexInput input = new ByteArrayIndexInput("indexrouting", indexRoutingStream.readAllBytes())) {
168+
long expectedChecksum;
169+
try {
170+
expectedChecksum = checksumOfChecksum(input.clone(), 8);
171+
} catch (Exception e) {
172+
throw e;
173+
}
174+
try (
175+
RemoteTransferContainer remoteTransferContainer = new RemoteTransferContainer(
176+
fileName,
177+
fileName,
178+
input.length(),
179+
true,
180+
WritePriority.URGENT,
181+
(size, position) -> new OffsetRangeIndexInputStream(input, size, position),
182+
expectedChecksum,
183+
((AsyncMultiStreamBlobContainer) blobContainer).remoteIntegrityCheckSupported()
184+
)
185+
) {
186+
return () -> ((AsyncMultiStreamBlobContainer) blobContainer).asyncBlobUpload(remoteTransferContainer.createWriteContext(), completionListener);
187+
} catch (IOException e) {
188+
e.printStackTrace();
189+
return null;
190+
}
191+
}
114192
}
115193

116-
public List<ClusterMetadataManifest.UploadedIndexMetadata> writeIncrementalRoutingTable(
117-
ClusterState previousClusterState,
118-
ClusterState clusterState,
119-
ClusterMetadataManifest previousManifest) {
120194

195+
public List<ClusterMetadataManifest.UploadedIndexMetadata> getAllUploadedIndicesRouting(ClusterMetadataManifest previousManifest, List<ClusterMetadataManifest.UploadedIndexMetadata> indicesRoutingToUpload, Set<String> indicesRoutingToDelete) {
121196
final Map<String, ClusterMetadataManifest.UploadedIndexMetadata> allUploadedIndicesRouting = previousManifest.getIndicesRouting()
122197
.stream()
123198
.collect(Collectors.toMap(ClusterMetadataManifest.UploadedIndexMetadata::getIndexName, Function.identity()));
199+
200+
indicesRoutingToUpload.forEach(
201+
uploadedIndexRouting -> allUploadedIndicesRouting.put(uploadedIndexRouting.getIndexName(), uploadedIndexRouting)
202+
);
203+
204+
indicesRoutingToDelete.forEach(index -> allUploadedIndicesRouting.remove(index));
205+
124206
logger.info("allUploadedIndicesRouting ROUTING {}", allUploadedIndicesRouting);
125207

126-
Map<String, IndexRoutingTable> previousIndexRoutingTable = previousClusterState.getRoutingTable().getIndicesRouting();
127-
List<ClusterMetadataManifest.UploadedIndexMetadata> uploadedIndices = new ArrayList<>();
128-
BlobPath custerMetadataBasePath = getCusterMetadataBasePath(blobStoreRepository, clusterState.getClusterName().value(),
129-
clusterState.metadata().clusterUUID());
130-
for (IndexRoutingTable indexRouting : clusterState.getRoutingTable().getIndicesRouting().values()) {
131-
if (previousIndexRoutingTable.containsKey(indexRouting.getIndex().getName()) && indexRouting.equals(previousIndexRoutingTable.get(indexRouting.getIndex().getName()))) {
132-
logger.info("index exists {}", indexRouting.getIndex().getName());
133-
//existing index with no shard change.
134-
uploadedIndices.add(allUploadedIndicesRouting.get(indexRouting.getIndex().getName()));
135-
} else {
136-
// new index or shards changed, in both cases we upload new index file.
137-
uploadedIndices.add(uploadIndex(indexRouting, custerMetadataBasePath));
138-
}
139-
}
140-
return uploadedIndices;
208+
return new ArrayList<>(allUploadedIndicesRouting.values());
141209
}
142210

143-
private ClusterMetadataManifest.UploadedIndexMetadata uploadIndex(IndexRoutingTable indexRouting, BlobPath custerMetadataBasePath) {
211+
private void uploadIndex(IndexRoutingTable indexRouting, String fileName, BlobContainer container) {
212+
logger.info("Starting write");
213+
144214
try {
145215
InputStream indexRoutingStream = new IndexRoutingTableInputStream(indexRouting);
146-
BlobContainer container = blobStoreRepository.blobStore().blobContainer(custerMetadataBasePath.add(INDEX_ROUTING_PATH_TOKEN).add(indexRouting.getIndex().getUUID()));
147-
String indexRoutingFileName = getIndexRoutingFileName();
148-
container.writeBlob(indexRoutingFileName, indexRoutingStream, 4096, true);
149-
return new ClusterMetadataManifest.UploadedIndexMetadata(indexRouting.getIndex().getName(), indexRouting.getIndex().getUUID(), container.path().buildAsString() + indexRoutingFileName);
150-
216+
container.writeBlob(fileName, indexRoutingStream, 4096, true);
217+
logger.info("SUccessful write");
151218
} catch (IOException e) {
152219
logger.error("Failed to write {}", e);
153220
}
154-
logger.info("SUccessful write");
155-
return null;
156221
}
157222

158223
private String getIndexRoutingFileName() {
159224
return String.join(
160225
DELIMITER,
161-
//RemoteStoreUtils.invertLong(indexMetadata.getVersion()),
162-
RemoteStoreUtils.invertLong(System.currentTimeMillis()),
163-
String.valueOf("CODEC1") // Keep the codec version at last place only, during read we reads last
164-
// place to determine codec version.
226+
INDEX_ROUTING_FILE_PREFIX,
227+
RemoteStoreUtils.invertLong(System.currentTimeMillis())
165228
);
166229

167230
}
168-
public RoutingTable getLatestRoutingTable(String clusterName, String clusterUUID) {
169-
return null;
170-
}
171231

172-
public RoutingTable getIncrementalRoutingTable(ClusterState previousClusterState, ClusterMetadataManifest previousManifest, String clusterName, String clusterUUID) {
173-
return null;
174-
}
175232

176233
public RoutingTable getIncrementalRoutingTable(ClusterState previousClusterState, ClusterMetadataManifest manifest){
177234
List<String> indicesRoutingDeleted = manifest.getDiffManifest().getIndicesRoutingDeleted();
@@ -232,9 +289,10 @@ public CheckedRunnable<IOException> getAsyncIndexMetadataReadAction(
232289
String uploadedFilename,
233290
Index index,
234291
LatchedActionListener<RemoteIndexRoutingResult> latchedActionListener) {
235-
BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer(getCusterMetadataBasePath(blobStoreRepository, clusterName, clusterUUID).add(INDEX_ROUTING_PATH_TOKEN).add(index.getUUID()));
236-
String[] fileNameTokens = uploadedFilename.split("/");
237-
String blobFileName = fileNameTokens[fileNameTokens.length -1];
292+
int idx = uploadedFilename.lastIndexOf("/");
293+
String blobFileName = uploadedFilename.substring(idx+1);
294+
BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer( BlobPath.cleanPath().add(uploadedFilename.substring(0,idx)));
295+
238296
return () -> readAsync(
239297
blobContainer,
240298
blobFileName,
@@ -262,8 +320,6 @@ public IndexRoutingTableInputStreamReader read(BlobContainer blobContainer, Stri
262320
}
263321
return null;
264322
}
265-
private void deleteStaleRoutingTable(String clusterName, String clusterUUID, int manifestsToRetain) {
266-
}
267323

268324
@Override
269325
public void close() throws IOException {
@@ -300,4 +356,5 @@ public IndexRoutingTable getIndexRoutingTable() {
300356
return indexRoutingTable;
301357
}
302358
}
359+
303360
}

0 commit comments

Comments
 (0)