diff --git a/CreateSnapshot/build.gradle b/CreateSnapshot/build.gradle index a3f3bb4b8..1b8c77dfb 100644 --- a/CreateSnapshot/build.gradle +++ b/CreateSnapshot/build.gradle @@ -15,7 +15,7 @@ dependencies { implementation project(':coreUtilities') implementation project(":RFS") - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'org.slf4j', name: 'slf4j-api' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' } diff --git a/CreateSnapshot/src/main/java/com/rfs/CreateSnapshot.java b/CreateSnapshot/src/main/java/com/rfs/CreateSnapshot.java index 85379c6f8..54f23c57d 100644 --- a/CreateSnapshot/src/main/java/com/rfs/CreateSnapshot.java +++ b/CreateSnapshot/src/main/java/com/rfs/CreateSnapshot.java @@ -1,7 +1,5 @@ package com.rfs; -import java.util.function.Function; - import org.opensearch.migrations.snapshot.creation.tracing.RootSnapshotContext; import org.opensearch.migrations.tracing.ActiveContextTracker; import org.opensearch.migrations.tracing.ActiveContextTrackerByActivityType; @@ -17,13 +15,14 @@ import com.rfs.common.OpenSearchClient; import com.rfs.common.S3SnapshotCreator; import com.rfs.common.SnapshotCreator; -import com.rfs.common.TryHandlePhaseFailure; import com.rfs.common.http.ConnectionContext; +import com.rfs.tracing.IRfsContexts.ICreateSnapshotContext; import com.rfs.worker.SnapshotRunner; import lombok.AllArgsConstructor; import lombok.Getter; import lombok.extern.slf4j.Slf4j; +@AllArgsConstructor @Slf4j public class CreateSnapshot { public static class Args { @@ -100,39 +99,44 @@ public static void main(String[] args) throws Exception { } log.info("Running CreateSnapshot with {}", String.join(" ", args)); - run( - c -> ((arguments.fileSystemRepoPath != null) - ? new FileSystemSnapshotCreator( + var snapshotCreator = new CreateSnapshot(arguments, rootContext.createSnapshotCreateContext()); + snapshotCreator.run(); + } + + private Args arguments; + private ICreateSnapshotContext context; + + public void run() { + var client = new OpenSearchClient(arguments.sourceArgs.toConnectionContext()); + SnapshotCreator snapshotCreator; + if (arguments.fileSystemRepoPath != null) { + snapshotCreator = new FileSystemSnapshotCreator( arguments.snapshotName, - c, + client, arguments.fileSystemRepoPath, - rootContext.createSnapshotCreateContext() - ) - : new S3SnapshotCreator( - arguments.snapshotName, - c, - arguments.s3RepoUri, - arguments.s3Region, - arguments.maxSnapshotRateMBPerNode, - arguments.s3RoleArn, - rootContext.createSnapshotCreateContext() - )), - new OpenSearchClient(arguments.sourceArgs.toConnectionContext()), - arguments.noWait - ); - } + context + ); + } else { + snapshotCreator = new S3SnapshotCreator( + arguments.snapshotName, + client, + arguments.s3RepoUri, + arguments.s3Region, + arguments.maxSnapshotRateMBPerNode, + arguments.s3RoleArn, + context + ); + } - public static void run( - Function snapshotCreatorFactory, - OpenSearchClient openSearchClient, - boolean noWait - ) throws Exception { - TryHandlePhaseFailure.executeWithTryCatch(() -> { - if (noWait) { - SnapshotRunner.run(snapshotCreatorFactory.apply(openSearchClient)); + try { + if (arguments.noWait) { + SnapshotRunner.run(snapshotCreator); } else { - SnapshotRunner.runAndWaitForCompletion(snapshotCreatorFactory.apply(openSearchClient)); + SnapshotRunner.runAndWaitForCompletion(snapshotCreator); } - }); + } catch (Exception e) { + log.atError().setMessage("Unexpected error running RfsWorker").setCause(e).log(); + throw e; + } } } diff --git a/CreateSnapshot/src/test/resources/log4j2.properties b/CreateSnapshot/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/CreateSnapshot/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/DocumentsFromSnapshotMigration/build.gradle b/DocumentsFromSnapshotMigration/build.gradle index 6967e76d9..369b90b47 100644 --- a/DocumentsFromSnapshotMigration/build.gradle +++ b/DocumentsFromSnapshotMigration/build.gradle @@ -33,7 +33,7 @@ dependencies { implementation group: 'org.apache.logging.log4j', name: 'log4j-api' implementation group: 'org.apache.logging.log4j', name: 'log4j-core' - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-annotations' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-core' diff --git a/DocumentsFromSnapshotMigration/src/main/java/com/rfs/RfsMigrateDocuments.java b/DocumentsFromSnapshotMigration/src/main/java/com/rfs/RfsMigrateDocuments.java index 08fd1cbda..a3d806ea0 100644 --- a/DocumentsFromSnapshotMigration/src/main/java/com/rfs/RfsMigrateDocuments.java +++ b/DocumentsFromSnapshotMigration/src/main/java/com/rfs/RfsMigrateDocuments.java @@ -6,11 +6,13 @@ import java.time.Clock; import java.time.Duration; import java.util.List; +import java.util.function.BiFunction; import java.util.function.Function; import org.opensearch.migrations.Version; import org.opensearch.migrations.VersionConverter; import org.opensearch.migrations.cluster.ClusterProviderRegistry; +import org.opensearch.migrations.cluster.ClusterSnapshotReader; import org.opensearch.migrations.reindexer.tracing.RootDocumentMigrationContext; import org.opensearch.migrations.tracing.ActiveContextTracker; import org.opensearch.migrations.tracing.ActiveContextTrackerByActivityType; @@ -23,6 +25,7 @@ import com.beust.jcommander.Parameter; import com.beust.jcommander.ParameterException; import com.beust.jcommander.ParametersDelegate; +import com.rfs.RfsMigrateDocuments.RunParameters; import com.rfs.cms.CoordinateWorkHttpClient; import com.rfs.cms.IWorkCoordinator; import com.rfs.cms.LeaseExpireTrigger; @@ -37,12 +40,13 @@ import com.rfs.common.S3Uri; import com.rfs.common.SnapshotShardUnpacker; import com.rfs.common.SourceRepo; -import com.rfs.common.TryHandlePhaseFailure; import com.rfs.common.http.ConnectionContext; import com.rfs.models.IndexMetadata; import com.rfs.models.ShardMetadata; import com.rfs.worker.DocumentsRunner; import com.rfs.worker.ShardWorkPreparer; +import lombok.Builder; +import lombok.NonNull; import lombok.extern.slf4j.Slf4j; import org.slf4j.MDC; @@ -167,7 +171,7 @@ public static void validateArgs(Args args) { } public static void main(String[] args) throws Exception { - System.err.println("Got args: " + String.join("; ", args)); + log.info("Got args: " + String.join("; ", args)); var workerId = ProcessHelpers.getNodeInstanceName(); log.info("Starting RfsMigrateDocuments with workerId =" + workerId); @@ -182,64 +186,63 @@ public static void main(String[] args) throws Exception { validateArgs(arguments); - var rootDocumentContext = makeRootContext(arguments, workerId); + var context = makeRootContext(arguments, workerId); var luceneDirPath = Paths.get(arguments.luceneDir); var snapshotLocalDirPath = arguments.snapshotLocalDir != null ? Paths.get(arguments.snapshotLocalDir) : null; + var connectionContext = arguments.targetArgs.toConnectionContext(); try (var processManager = new LeaseExpireTrigger(workItemId -> { log.error("Terminating RfsMigrateDocuments because the lease has expired for " + workItemId); System.exit(PROCESS_TIMED_OUT); - }, Clock.systemUTC())) { - ConnectionContext connectionContext = arguments.targetArgs.toConnectionContext(); + }, Clock.systemUTC()); var workCoordinator = new OpenSearchWorkCoordinator( new CoordinateWorkHttpClient(connectionContext), TOLERABLE_CLIENT_SERVER_CLOCK_DIFFERENCE_SECONDS, workerId - ); + )) { MDC.put(LOGGING_MDC_WORKER_ID, workerId); // I don't see a need to clean this up since we're in main - TryHandlePhaseFailure.executeWithTryCatch(() -> { - OpenSearchClient targetClient = new OpenSearchClient(connectionContext); - DocumentReindexer reindexer = new DocumentReindexer(targetClient, - arguments.numDocsPerBulkRequest, - arguments.numBytesPerBulkRequest, - arguments.maxConnections); - - SourceRepo sourceRepo; - if (snapshotLocalDirPath == null) { - sourceRepo = S3Repo.create( - Paths.get(arguments.s3LocalDir), - new S3Uri(arguments.s3RepoUri), - arguments.s3Region - ); - } else { - sourceRepo = new FileSystemRepo(snapshotLocalDirPath); - } - DefaultSourceRepoAccessor repoAccessor = new DefaultSourceRepoAccessor(sourceRepo); - - var sourceResourceProvider = ClusterProviderRegistry.getSnapshotReader(arguments.sourceVersion, sourceRepo); - - SnapshotShardUnpacker.Factory unpackerFactory = new SnapshotShardUnpacker.Factory( - repoAccessor, - luceneDirPath, - sourceResourceProvider.getBufferSizeInBytes() + OpenSearchClient targetClient = new OpenSearchClient(connectionContext); + DocumentReindexer reindexer = new DocumentReindexer(targetClient, + arguments.numDocsPerBulkRequest, + arguments.numBytesPerBulkRequest, + arguments.maxConnections); + + SourceRepo sourceRepo; + if (snapshotLocalDirPath == null) { + sourceRepo = S3Repo.create( + Paths.get(arguments.s3LocalDir), + new S3Uri(arguments.s3RepoUri), + arguments.s3Region ); + } else { + sourceRepo = new FileSystemRepo(snapshotLocalDirPath); + } + var repoAccessor = new DefaultSourceRepoAccessor(sourceRepo); - run( - LuceneDocumentsReader.getFactory(sourceResourceProvider.getSoftDeletesPossible(), - sourceResourceProvider.getSoftDeletesFieldData()), - reindexer, - workCoordinator, - arguments.initialLeaseDuration, - processManager, - sourceResourceProvider.getIndexMetadata(), - arguments.snapshotName, - arguments.indexAllowlist, - sourceResourceProvider.getShardMetadata(), - unpackerFactory, - arguments.maxShardSizeBytes, - rootDocumentContext - ); - }); + var sourceResourceProvider = ClusterProviderRegistry.getSnapshotReader(arguments.sourceVersion, sourceRepo); + + var unpackerFactory = new SnapshotShardUnpacker.Factory( + repoAccessor, + luceneDirPath, + sourceResourceProvider.getBufferSizeInBytes() + ); + + run(RunParameters.builder() + .leaseExpireTrigger(processManager) + .workCoordinator(workCoordinator) + .reindexer(reindexer) + .snapshotName(arguments.snapshotName) + .snapshotReader(sourceResourceProvider) + .snapshotUnpacker(unpackerFactory) + .documentReader(LuceneDocumentsReader.getFactory(sourceResourceProvider)) + .indexAllowlist(arguments.indexAllowlist) + .maxInitialLeaseDuration(arguments.initialLeaseDuration) + .maxShardSizeBytes(arguments.maxShardSizeBytes) + .tracingContext(context) + .build()); + } catch (Exception e) { + log.atError().setMessage("Unexpected error running RfsWorker").setCause(e).log(); + throw e; } } @@ -256,41 +259,38 @@ private static RootDocumentMigrationContext makeRootContext(Args arguments, Stri return new RootDocumentMigrationContext(otelSdk, compositeContextTracker); } - public static DocumentsRunner.CompletionStatus run( - Function readerFactory, - DocumentReindexer reindexer, - IWorkCoordinator workCoordinator, - Duration maxInitialLeaseDuration, - LeaseExpireTrigger leaseExpireTrigger, - IndexMetadata.Factory indexMetadataFactory, - String snapshotName, - List indexAllowlist, - ShardMetadata.Factory shardMetadataFactory, - SnapshotShardUnpacker.Factory unpackerFactory, - long maxShardSizeBytes, - RootDocumentMigrationContext rootDocumentContext - ) throws IOException, InterruptedException, NoWorkLeftException { - var scopedWorkCoordinator = new ScopedWorkCoordinator(workCoordinator, leaseExpireTrigger); + public static DocumentsRunner.CompletionStatus run(RunParameters params) throws Exception { + var scopedWorkCoordinator = new ScopedWorkCoordinator(params.workCoordinator, params.leaseExpireTrigger); confirmShardPrepIsComplete( - indexMetadataFactory, - snapshotName, - indexAllowlist, + params.snapshotReader.getIndexMetadata(), + params.snapshotName, + params.indexAllowlist, scopedWorkCoordinator, - rootDocumentContext + params.tracingContext ); - if (!workCoordinator.workItemsArePending( - rootDocumentContext.getWorkCoordinationContext()::createItemsPendingContext + if (!params.workCoordinator.workItemsArePending( + params.tracingContext.getWorkCoordinationContext()::createItemsPendingContext )) { throw new NoWorkLeftException("No work items are pending/all work items have been processed. Returning."); } - return new DocumentsRunner(scopedWorkCoordinator, maxInitialLeaseDuration, (name, shard) -> { - var shardMetadata = shardMetadataFactory.fromRepo(snapshotName, name, shard); + BiFunction shardFactory = (name, shard) -> { + var shardMetadataFactory = params.snapshotReader.getShardMetadata(); + var shardMetadata = shardMetadataFactory.fromRepo(params.snapshotName, name, shard); log.info("Shard size: " + shardMetadata.getTotalSizeBytes()); - if (shardMetadata.getTotalSizeBytes() > maxShardSizeBytes) { - throw new DocumentsRunner.ShardTooLargeException(shardMetadata.getTotalSizeBytes(), maxShardSizeBytes); + if (shardMetadata.getTotalSizeBytes() > params.maxShardSizeBytes) { + throw new DocumentsRunner.ShardTooLargeException(shardMetadata.getTotalSizeBytes(), params.maxShardSizeBytes); } return shardMetadata; - }, unpackerFactory, readerFactory, reindexer).migrateNextShard(rootDocumentContext::createReindexContext); + }; + var runner = new DocumentsRunner( + scopedWorkCoordinator, + params.maxInitialLeaseDuration, + shardFactory, + params.snapshotUnpacker, + params.documentReader, + params.reindexer); + var migrationStatus = runner.migrateNextShard(params.tracingContext::createReindexContext); + return migrationStatus; } private static void confirmShardPrepIsComplete( @@ -330,8 +330,32 @@ private static void confirmShardPrepIsComplete( .log(); Thread.sleep(lockRenegotiationMillis); lockRenegotiationMillis *= 2; - continue; } } } + + @Builder + static class RunParameters { + @NonNull + final LeaseExpireTrigger leaseExpireTrigger; + @NonNull + final IWorkCoordinator workCoordinator; + @NonNull + final String snapshotName; + @NonNull + final ClusterSnapshotReader snapshotReader; + @NonNull + final SnapshotShardUnpacker.Factory snapshotUnpacker; + @NonNull + final Function documentReader; + @NonNull + final DocumentReindexer reindexer; + @NonNull + final List indexAllowlist; + @NonNull + final Duration maxInitialLeaseDuration; + final long maxShardSizeBytes; + @NonNull + final RootDocumentMigrationContext tracingContext; + } } diff --git a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/EndToEndTest.java b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/EndToEndTest.java index 7f338bab7..0ad7f1846 100644 --- a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/EndToEndTest.java +++ b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/EndToEndTest.java @@ -5,12 +5,11 @@ import java.util.Random; import java.util.concurrent.CompletableFuture; -import org.hamcrest.Matchers; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ArgumentsSource; -import org.opensearch.migrations.metadata.tracing.MetadataMigrationTestContext; import org.opensearch.migrations.reindexer.tracing.DocumentMigrationTestContext; import org.opensearch.migrations.snapshot.creation.tracing.SnapshotTestContext; @@ -27,6 +26,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +@Tag("isolatedTest") public class EndToEndTest extends SourceTestBase { @TempDir private File localDirectory; @@ -34,90 +34,11 @@ public class EndToEndTest extends SourceTestBase { @ParameterizedTest(name = "Target {0}") @ArgumentsSource(SupportedTargetCluster.class) public void migrateFrom_ES_v6_8(final SearchClusterContainer.ContainerVersion targetVersion) throws Exception { - final var snapshotContext = SnapshotTestContext.factory().noOtelTracking(); - final var metadataContext = MetadataMigrationTestContext.factory().noOtelTracking(); - final var testDocMigrationContext = DocumentMigrationTestContext.factory().noOtelTracking(); - try ( final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V6_8_23); final var targetCluster = new SearchClusterContainer(targetVersion) ) { - // === ACTION: Set up the source/target clusters === - var bothClustersStarted = CompletableFuture.allOf( - CompletableFuture.runAsync(() -> sourceCluster.start()), - CompletableFuture.runAsync(() -> targetCluster.start()) - ); - bothClustersStarted.join(); - - // Create a template - var sourceClusterOperations = new ClusterOperations(sourceCluster.getUrl()); - var templateName = "my_template_foo"; - sourceClusterOperations.createES6LegacyTemplate(templateName, "bar*"); - var indexName = "barstool"; - - // Create a document that uses the template - sourceClusterOperations.createDocument(indexName, "222", "{\"hi\":\"yay\"}"); - - // === ACTION: Take a snapshot === - var snapshotName = "my_snap"; - var sourceClient = new OpenSearchClient(ConnectionContextTestParams.builder() - .host(sourceCluster.getUrl()) - .insecure(true) - .build() - .toConnectionContext()); - var snapshotCreator = new FileSystemSnapshotCreator( - snapshotName, - sourceClient, - SearchClusterContainer.CLUSTER_SNAPSHOT_DIR, - snapshotContext.createSnapshotCreateContext() - ); - SnapshotRunner.runAndWaitForCompletion(snapshotCreator); - sourceCluster.copySnapshotData(localDirectory.toString()); - - var sourceRepo = new FileSystemRepo(localDirectory.toPath()); - var targetClient = new OpenSearchClient(ConnectionContextTestParams.builder() - .host(targetCluster.getUrl()) - .insecure(true) - .build() - .toConnectionContext()); - - // === ACTION: Migrate the templates and indices === - migrateMetadata( - sourceRepo, - targetClient, - snapshotName, - List.of(templateName), - List.of(), - List.of(), - List.of(), - metadataContext, - sourceCluster.getContainerVersion().getVersion() - ); - - // Check that the templates were migrated - var targetClusterOperations = new ClusterOperations(targetCluster.getUrl()); - var res = targetClusterOperations.get("/_template/" + templateName); - assertThat(res.getValue(), res.getKey(), equalTo(200)); - assertThat(res.getValue(), Matchers.containsString("mappings\":{")); - - // === ACTION: Migrate the documents === - final var clockJitter = new Random(1); - var result = migrateDocumentsWithOneWorker( - sourceRepo, - snapshotName, - List.of(), - targetCluster.getUrl(), - clockJitter, - testDocMigrationContext, - sourceCluster.getContainerVersion().getVersion(), - false - ); - assertThat(result, equalTo(DocumentsRunner.CompletionStatus.WORK_COMPLETED)); - - // Check that the docs were migrated - checkClusterMigrationOnFinished(sourceCluster, targetCluster, testDocMigrationContext); - } finally { - deleteTree(localDirectory.toPath()); + migrateFrom_ES(sourceCluster, targetCluster); } } @@ -128,7 +49,7 @@ public void migrateFrom_ES_v7_10(final SearchClusterContainer.ContainerVersion t final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V7_10_2); final var targetCluster = new SearchClusterContainer(targetVersion) ) { - migrateFrom_ES_v7_X(sourceCluster, targetCluster); + migrateFrom_ES(sourceCluster, targetCluster); } } @@ -139,7 +60,7 @@ public void migrateFrom_ES_v7_17(final SearchClusterContainer.ContainerVersion t final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V7_17); final var targetCluster = new SearchClusterContainer(targetVersion) ) { - migrateFrom_ES_v7_X(sourceCluster, targetCluster); + migrateFrom_ES(sourceCluster, targetCluster); } } @@ -150,17 +71,16 @@ public void migrateFrom_OS_v1_3(final SearchClusterContainer.ContainerVersion ta final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.OS_V1_3_16); final var targetCluster = new SearchClusterContainer(targetVersion) ) { - migrateFrom_ES_v7_X(sourceCluster, targetCluster); + migrateFrom_ES(sourceCluster, targetCluster); } } @SneakyThrows - private void migrateFrom_ES_v7_X( + private void migrateFrom_ES( final SearchClusterContainer sourceCluster, final SearchClusterContainer targetCluster ) { final var snapshotContext = SnapshotTestContext.factory().noOtelTracking(); - final var metadataContext = MetadataMigrationTestContext.factory().noOtelTracking(); final var testDocMigrationContext = DocumentMigrationTestContext.factory().noOtelTracking(); try { @@ -172,14 +92,8 @@ private void migrateFrom_ES_v7_X( ); bothClustersStarted.join(); - // Create the component and index templates - var sourceClusterOperations = new ClusterOperations(sourceCluster.getUrl()); - var compoTemplateName = "simple_component_template"; - var indexTemplateName = "simple_index_template"; - sourceClusterOperations.createES7Templates(compoTemplateName, indexTemplateName, "author", "blog*"); var indexName = "blog_2023"; - - // Creates a document that uses the template + var sourceClusterOperations = new ClusterOperations(sourceCluster.getUrl()); sourceClusterOperations.createDocument(indexName, "222", "{\"author\":\"Tobias Funke\"}"); // === ACTION: Take a snapshot === @@ -197,32 +111,7 @@ private void migrateFrom_ES_v7_X( ); SnapshotRunner.runAndWaitForCompletion(snapshotCreator); sourceCluster.copySnapshotData(localDirectory.toString()); - var sourceRepo = new FileSystemRepo(localDirectory.toPath()); - var targetClient = new OpenSearchClient(ConnectionContextTestParams.builder() - .host(targetCluster.getUrl()) - .insecure(true) - .build() - .toConnectionContext()); - - // === ACTION: Migrate the templates and indices === - migrateMetadata( - sourceRepo, - targetClient, - snapshotName, - List.of(), - List.of(compoTemplateName), - List.of(indexTemplateName), - List.of(), - metadataContext, - sourceCluster.getContainerVersion().getVersion() - ); - - // Check that the templates were migrated - var targetClusterOperations = new ClusterOperations(targetCluster.getUrl()); - var res = targetClusterOperations.get("/_index_template/" + indexTemplateName); - assertThat(res.getValue(), res.getKey(), equalTo(200)); - assertThat(res.getValue(), Matchers.containsString("composed_of\":[\"" + compoTemplateName + "\"]")); // === ACTION: Migrate the documents === final var clockJitter = new Random(1); diff --git a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ParallelDocumentMigrationsTest.java b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ParallelDocumentMigrationsTest.java index 093f6360f..e846ce7c3 100644 --- a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ParallelDocumentMigrationsTest.java +++ b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ParallelDocumentMigrationsTest.java @@ -20,20 +20,16 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.opensearch.migrations.metadata.tracing.MetadataMigrationTestContext; import org.opensearch.migrations.reindexer.tracing.DocumentMigrationTestContext; import org.opensearch.migrations.snapshot.creation.tracing.SnapshotTestContext; import com.rfs.common.FileSystemRepo; -import com.rfs.common.FileSystemSnapshotCreator; -import com.rfs.common.OpenSearchClient; -import com.rfs.common.http.ConnectionContextTestParams; import com.rfs.framework.PreloadedSearchClusterContainer; import com.rfs.framework.SearchClusterContainer; import lombok.Lombok; import lombok.extern.slf4j.Slf4j; -@Tag("longTest") +@Tag("isolatedTest") @Slf4j public class ParallelDocumentMigrationsTest extends SourceTestBase { @@ -80,7 +76,6 @@ public void testDocumentMigration( ) throws Exception { var executorService = Executors.newFixedThreadPool(numWorkers); final var testSnapshotContext = SnapshotTestContext.factory().noOtelTracking(); - final var testMetadataMigrationContext = MetadataMigrationTestContext.factory().noOtelTracking(); final var testDocMigrationContext = DocumentMigrationTestContext.factory() .withAllTracking(); @@ -101,32 +96,20 @@ public void testDocumentMigration( return null; }, executorService)).join(); - final var SNAPSHOT_NAME = "test_snapshot"; + var args = new CreateSnapshot.Args(); + args.snapshotName = "test_snapshot"; + args.fileSystemRepoPath = SearchClusterContainer.CLUSTER_SNAPSHOT_DIR; + args.sourceArgs.host = esSourceContainer.getUrl(); + + var snapshotCreator = new CreateSnapshot(args, testSnapshotContext.createSnapshotCreateContext()); + snapshotCreator.run(); + + final List INDEX_ALLOWLIST = List.of(); - CreateSnapshot.run( - c -> new FileSystemSnapshotCreator( - SNAPSHOT_NAME, - c, - SearchClusterContainer.CLUSTER_SNAPSHOT_DIR, - testSnapshotContext.createSnapshotCreateContext() - ), - new OpenSearchClient(ConnectionContextTestParams.builder() - .host(esSourceContainer.getUrl()) - .build() - .toConnectionContext()), - false - ); var tempDir = Files.createTempDirectory("opensearchMigrationReindexFromSnapshot_test_snapshot"); try { esSourceContainer.copySnapshotData(tempDir.toString()); - - var targetClient = new OpenSearchClient(ConnectionContextTestParams.builder() - .host(esSourceContainer.getUrl()) - .build() - .toConnectionContext()); var sourceRepo = new FileSystemRepo(tempDir); - migrateMetadata(sourceRepo, targetClient, SNAPSHOT_NAME, List.of(), List.of(), List.of(), INDEX_ALLOWLIST, testMetadataMigrationContext, baseSourceImageVersion.getVersion()); - var workerFutures = new ArrayList>(); var runCounter = new AtomicInteger(); final var clockJitter = new Random(1); @@ -136,7 +119,7 @@ public void testDocumentMigration( CompletableFuture.supplyAsync( () -> migrateDocumentsSequentially( sourceRepo, - SNAPSHOT_NAME, + args.snapshotName, INDEX_ALLOWLIST, osTargetContainer.getUrl(), runCounter, diff --git a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/PerformanceVerificationTest.java b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/PerformanceVerificationTest.java index 1d1d26e24..6d34df0a2 100644 --- a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/PerformanceVerificationTest.java +++ b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/PerformanceVerificationTest.java @@ -13,6 +13,7 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.store.ByteBuffersDirectory; import org.apache.lucene.util.BytesRef; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.opensearch.migrations.reindexer.tracing.IDocumentMigrationContexts; @@ -21,6 +22,7 @@ import com.rfs.common.LuceneDocumentsReader; import com.rfs.common.OpenSearchClient; import com.rfs.common.OpenSearchClient.BulkResponse; +import com.rfs.common.RfsLuceneDocument; import com.rfs.tracing.IRfsContexts; import lombok.extern.slf4j.Slf4j; import reactor.core.publisher.Mono; @@ -37,6 +39,7 @@ public class PerformanceVerificationTest { @Test + @Tag("isolatedTest") void testDocumentBuffering() throws Exception { // Create an in-memory directory for the test ByteBuffersDirectory inMemoryDir = new ByteBuffersDirectory(); @@ -68,7 +71,7 @@ protected DirectoryReader getReader() { } @Override - protected Document getDocument(IndexReader reader, int docId, boolean isLive) { + protected RfsLuceneDocument getDocument(IndexReader reader, int docId, boolean isLive) { ingestedDocuments.incrementAndGet(); return super.getDocument(reader, docId, isLive); } diff --git a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ProcessLifecycleTest.java b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ProcessLifecycleTest.java index 534cae4a8..1c06d167e 100644 --- a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ProcessLifecycleTest.java +++ b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/ProcessLifecycleTest.java @@ -16,16 +16,10 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; -import org.opensearch.migrations.Version; -import org.opensearch.migrations.metadata.tracing.MetadataMigrationTestContext; import org.opensearch.migrations.snapshot.creation.tracing.SnapshotTestContext; import org.opensearch.migrations.testutils.ToxiProxyWrapper; import org.opensearch.testcontainers.OpensearchContainer; -import com.rfs.common.FileSystemRepo; -import com.rfs.common.FileSystemSnapshotCreator; -import com.rfs.common.OpenSearchClient; -import com.rfs.common.http.ConnectionContextTestParams; import com.rfs.framework.PreloadedSearchClusterContainer; import com.rfs.framework.SearchClusterContainer; import eu.rekawek.toxiproxy.model.ToxicDirection; @@ -68,7 +62,6 @@ enum FailHow { public void testProcessExitsAsExpected(String failAfterString, int expectedExitCode) throws Exception { final var failHow = FailHow.valueOf(failAfterString); final var testSnapshotContext = SnapshotTestContext.factory().noOtelTracking(); - final var testMetadataMigrationContext = MetadataMigrationTestContext.factory().noOtelTracking(); var sourceImageArgs = makeParamsForBase(SearchClusterContainer.ES_V7_10_2); var baseSourceImageVersion = (SearchClusterContainer.ContainerVersion) sourceImageArgs[0]; @@ -104,22 +97,15 @@ public void testProcessExitsAsExpected(String failAfterString, int expectedExitC return null; })).join(); - CreateSnapshot.run( - c -> new FileSystemSnapshotCreator( - SNAPSHOT_NAME, - c, - SearchClusterContainer.CLUSTER_SNAPSHOT_DIR, - testSnapshotContext.createSnapshotCreateContext() - ), - new OpenSearchClient(ConnectionContextTestParams.builder() - .host(esSourceContainer.getUrl()) - .build() - .toConnectionContext()), - false - ); - esSourceContainer.copySnapshotData(tempDirSnapshot.toString()); + var args = new CreateSnapshot.Args(); + args.snapshotName = SNAPSHOT_NAME; + args.fileSystemRepoPath = SearchClusterContainer.CLUSTER_SNAPSHOT_DIR; + args.sourceArgs.host = esSourceContainer.getUrl(); - migrateMetadata(osTargetContainer, tempDirSnapshot, testMetadataMigrationContext, baseSourceImageVersion.getVersion()); + var snapshotCreator = new CreateSnapshot(args, testSnapshotContext.createSnapshotCreateContext()); + snapshotCreator.run(); + + esSourceContainer.copySnapshotData(tempDirSnapshot.toString()); int actualExitCode = runProcessAgainstToxicTarget(tempDirSnapshot, tempDirLucene, proxyContainer, failHow); log.atInfo().setMessage("Process exited with code: " + actualExitCode).log(); @@ -136,24 +122,6 @@ public void testProcessExitsAsExpected(String failAfterString, int expectedExitC } } - private static void migrateMetadata( - OpensearchContainer targetContainer, - Path tempDirSnapshot, - MetadataMigrationTestContext testMetadataMigrationContext, - Version sourceVersion - ) { - String targetAddress = "http://" - + targetContainer.getHost() - + ":" - + targetContainer.getMappedPort(OPENSEARCH_PORT); - var targetClient = new OpenSearchClient(ConnectionContextTestParams.builder() - .host(targetAddress) - .build() - .toConnectionContext()); - var sourceRepo = new FileSystemRepo(tempDirSnapshot); - migrateMetadata(sourceRepo, targetClient, SNAPSHOT_NAME, List.of(), List.of(), List.of(), INDEX_ALLOWLIST, testMetadataMigrationContext, sourceVersion); - } - private static int runProcessAgainstToxicTarget( Path tempDirSnapshot, Path tempDirLucene, diff --git a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/SourceTestBase.java b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/SourceTestBase.java index 794ce3e90..5fed0ee79 100644 --- a/DocumentsFromSnapshotMigration/src/test/java/com/rfs/SourceTestBase.java +++ b/DocumentsFromSnapshotMigration/src/test/java/com/rfs/SourceTestBase.java @@ -15,16 +15,15 @@ import java.util.function.Function; import java.util.function.UnaryOperator; -import org.apache.lucene.document.Document; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; import org.junit.jupiter.api.Assertions; import org.opensearch.migrations.Version; import org.opensearch.migrations.cluster.ClusterProviderRegistry; -import org.opensearch.migrations.metadata.tracing.MetadataMigrationTestContext; import org.opensearch.migrations.reindexer.tracing.DocumentMigrationTestContext; +import com.rfs.RfsMigrateDocuments.RunParameters; import com.rfs.cms.CoordinateWorkHttpClient; import com.rfs.cms.LeaseExpireTrigger; import com.rfs.cms.OpenSearchWorkCoordinator; @@ -34,18 +33,13 @@ import com.rfs.common.LuceneDocumentsReader; import com.rfs.common.OpenSearchClient; import com.rfs.common.RestClient; +import com.rfs.common.RfsLuceneDocument; import com.rfs.common.SnapshotShardUnpacker; import com.rfs.common.SourceRepo; import com.rfs.common.http.ConnectionContextTestParams; import com.rfs.framework.SearchClusterContainer; import com.rfs.http.SearchClusterRequests; -import com.rfs.transformers.TransformFunctions; -import com.rfs.transformers.Transformer; -import com.rfs.version_os_2_11.GlobalMetadataCreator_OS_2_11; -import com.rfs.version_os_2_11.IndexCreator_OS_2_11; import com.rfs.worker.DocumentsRunner; -import com.rfs.worker.IndexRunner; -import com.rfs.worker.MetadataRunner; import lombok.AllArgsConstructor; import lombok.Lombok; import lombok.SneakyThrows; @@ -66,43 +60,6 @@ protected static Object[] makeParamsForBase(SearchClusterContainer.ContainerVers new String[] { "/root/runTestBenchmarks.sh", "--endpoint", "http://" + SOURCE_SERVER_ALIAS + ":9200/" } }; } - protected static void migrateMetadata( - SourceRepo sourceRepo, - OpenSearchClient targetClient, - String snapshotName, - List legacyTemplateAllowlist, - List componentTemplateAllowlist, - List indexTemplateAllowlist, - List indexAllowlist, - MetadataMigrationTestContext context, - Version sourceVersion - ) { - var sourceResourceProvider = ClusterProviderRegistry.getSnapshotReader(sourceVersion, sourceRepo); - var targetVersion = Version.fromString("OS 2.11"); - GlobalMetadataCreator_OS_2_11 metadataCreator = new GlobalMetadataCreator_OS_2_11( - targetClient, - legacyTemplateAllowlist, - componentTemplateAllowlist, - indexTemplateAllowlist - ); - Transformer transformer = TransformFunctions.getTransformer(sourceVersion, targetVersion, 1); - new MetadataRunner( - snapshotName, - sourceResourceProvider.getGlobalMetadata(), - metadataCreator, - transformer - ).migrateMetadata(context.createMetadataMigrationContext()); - - IndexCreator_OS_2_11 indexCreator = new IndexCreator_OS_2_11(targetClient); - new IndexRunner( - snapshotName, - sourceResourceProvider.getIndexMetadata(), - indexCreator, - transformer, - indexAllowlist - ).migrateIndices(context.createIndexContext()); - } - @AllArgsConstructor public static class ExpectedMigrationWorkTerminationException extends RuntimeException { public final RfsMigrateDocuments.NoWorkLeftException exception; @@ -181,15 +138,15 @@ public static int migrateDocumentsSequentially( } public static class FilteredLuceneDocumentsReader extends LuceneDocumentsReader { - private final UnaryOperator docTransformer; + private final UnaryOperator docTransformer; - public FilteredLuceneDocumentsReader(Path luceneFilesBasePath, boolean softDeletesPossible, String softDeletesField, UnaryOperator docTransformer) { + public FilteredLuceneDocumentsReader(Path luceneFilesBasePath, boolean softDeletesPossible, String softDeletesField, UnaryOperator docTransformer) { super(luceneFilesBasePath, softDeletesPossible, softDeletesField); this.docTransformer = docTransformer; } @Override - public Flux readDocuments() { + public Flux readDocuments() { return super.readDocuments().map(docTransformer::apply); } } @@ -213,7 +170,7 @@ public static DocumentsRunner.CompletionStatus migrateDocumentsWithOneWorker( log.atDebug().setMessage("Lease expired for " + workItemId + " making next document get throw").log(); shouldThrow.set(true); })) { - UnaryOperator terminatingDocumentFilter = d -> { + UnaryOperator terminatingDocumentFilter = d -> { if (shouldThrow.get()) { throw new LeasePastError(); } @@ -233,36 +190,37 @@ public static DocumentsRunner.CompletionStatus migrateDocumentsWithOneWorker( final var nextClockShift = (int) (clockJitter.nextDouble() * ms_window) - (ms_window / 2); log.info("nextClockShift=" + nextClockShift); - Function readerFactory = path -> new FilteredLuceneDocumentsReader(path, sourceResourceProvider.getSoftDeletesPossible(), sourceResourceProvider.getSoftDeletesFieldData(), terminatingDocumentFilter); - return RfsMigrateDocuments.run( - readerFactory, - new DocumentReindexer(new OpenSearchClient(ConnectionContextTestParams.builder() + try (var workCoordinator = new OpenSearchWorkCoordinator( + new CoordinateWorkHttpClient(ConnectionContextTestParams.builder() .host(targetAddress) - .compressionEnabled(compressionEnabled) .build() - .toConnectionContext()), 1000, Long.MAX_VALUE, 1), - new OpenSearchWorkCoordinator( - new CoordinateWorkHttpClient(ConnectionContextTestParams.builder() + .toConnectionContext()), + TOLERABLE_CLIENT_SERVER_CLOCK_DIFFERENCE_SECONDS, + UUID.randomUUID().toString(), + Clock.offset(Clock.systemUTC(), Duration.ofMillis(nextClockShift)) + )) { + return RfsMigrateDocuments.run(RunParameters.builder() + .leaseExpireTrigger(processManager) + .workCoordinator(workCoordinator) + .snapshotName(snapshotName) + .snapshotReader(sourceResourceProvider) + .snapshotUnpacker(unpackerFactory) + .documentReader(readerFactory) + .reindexer(new DocumentReindexer(new OpenSearchClient(ConnectionContextTestParams.builder() .host(targetAddress) + .compressionEnabled(compressionEnabled) .build() - .toConnectionContext()), - TOLERABLE_CLIENT_SERVER_CLOCK_DIFFERENCE_SECONDS, - UUID.randomUUID().toString(), - Clock.offset(Clock.systemUTC(), Duration.ofMillis(nextClockShift)) - ), - Duration.ofMinutes(10), - processManager, - sourceResourceProvider.getIndexMetadata(), - snapshotName, - indexAllowlist, - sourceResourceProvider.getShardMetadata(), - unpackerFactory, - MAX_SHARD_SIZE_BYTES, - context - ); + .toConnectionContext()), 1000, Long.MAX_VALUE, 1)) + .maxInitialLeaseDuration(Duration.ofMinutes(10)) + .indexAllowlist(indexAllowlist) + .maxShardSizeBytes(MAX_SHARD_SIZE_BYTES) + .tracingContext(context) + .build() + ); + } } finally { deleteTree(tempDir); } diff --git a/DocumentsFromSnapshotMigration/src/test/resources/log4j2.properties b/DocumentsFromSnapshotMigration/src/test/resources/log4j2.properties index 1f9ecf437..a389da4cd 100644 --- a/DocumentsFromSnapshotMigration/src/test/resources/log4j2.properties +++ b/DocumentsFromSnapshotMigration/src/test/resources/log4j2.properties @@ -1,7 +1,8 @@ -status = ERROR +status = WARN property.logsDir = ${env:SHARED_LOGS_DIR_PATH:-./logs} property.failedLoggerFileNamePrefix = ${logsDir}/${hostName}/failedRequests/failedRequests +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} appenders = console, FailedRequests @@ -26,11 +27,17 @@ appender.console.type = Console appender.console.name = Console appender.console.target = SYSTEM_OUT appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{HH:mm:ss.SSS} %threadName %-5p %c{1}:%L - %m%n +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n rootLogger.level = info rootLogger.appenderRef.console.ref = Console +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} + logger.wireLogger.name = org.apache.http.wire logger.wireLogger.level = OFF -logger.wireLogger.additivity = false \ No newline at end of file +logger.wireLogger.additivity = false diff --git a/MetadataMigration/DEVELOPER_GUIDE.md b/MetadataMigration/DEVELOPER_GUIDE.md new file mode 100644 index 000000000..24544d2ff --- /dev/null +++ b/MetadataMigration/DEVELOPER_GUIDE.md @@ -0,0 +1,83 @@ +## Metadata Migration Developer Guide + +- [Metadata Migration Developer Guide](#metadata-migration-developer-guide) +- [Architecture](#architecture) +- [How to run tests](#how-to-run-tests) +- [How to use the tool interactively](#how-to-use-the-tool-interactively) + - [S3 Snapshot](#s3-snapshot) + - [On-Disk Snapshot](#on-disk-snapshot) + - [Handling Auth](#handling-auth) + - [Allowlisting the templates and indices to migrate](#allowlisting-the-templates-and-indices-to-migrate) + +## Architecture + +The Metadata migration project holds classes that is specific only to the CLI tool and end to end test cases for the CLI. The majority of the business logic is in the RFS library as it is shared between [CreateSnapshot](../CreateSnapshot/README.md) and [DocumentFromSnapshotMigration](../DocumentsFromSnapshotMigration/README.md) tools. + +```mermaid +graph LR + mm[MetadataMigration] --> |Data models, Cluster Readers/Writers| RFS + mm --> |Transformation logic| t[transformation] + mm --> |Authentication, Telemetry| cu[coreUtilities] + mm --> |AWS functionality|au[awsUtilities] +``` + +## How to run tests + +Runs all unit test cases, very fast for running unit tests +```shell +./gradlew MetadataMigration:test +``` + +Run all test cases include starting up docker images for end to end verification, runtime will be at least 5 minutes. + +```shell +./gradlew MetadataMigration:slowTest +``` + +(Often Used) Run end to end test cases for only a single source platform, tests all commands with a runtime of ~1 minute. +```shell +./gradlew MetadataMigration:slowTest --tests *metadataMigrateFrom_OS_v1_3*` +``` + +## How to use the tool interactively + +You can kick off the locally tool using Gradle. + +### S3 Snapshot + +From the root directory of the repo, run a CLI command like so: + +```shell +./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200' +``` + +In order for this succeed, you'll need to make sure you have valid AWS Credentials in your key ring (~/.aws/credentials) with permission to operate on the S3 URI specified. + +### On-Disk Snapshot + +From the root directory of the repo, run a CLI command like so: + +```shell +./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --file-system-repo-path /snapshot --s3-region us-fake-1 --target-host http://hostname:9200' +``` + +### Handling Auth + +If your target cluster has basic auth enabled on it, you can supply those credentials to the tool via the CLI: + +```shell +./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200 --target-username --target-password ' +``` + +### Allowlisting the templates and indices to migrate + +By default, the tool has an empty allowlist for templates, meaning none will be migrated. In contrast, the default allowlist for indices is open, meaning all non-system indices (those not prefixed with `.`) will be migrated. You can tweak these allowlists with a comma-separated list of items you specifically with to migrate. If you specify an custom allowlist for the templates or indices, the default allowlist is disregarded and **only** the items you have in your allowlist will be moved. + +```shell +./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200 --index-allowlist Index1,.my_system_index,logs-2023 --index-template-allowlist logs_template --component-template-allowlist component2,component7' +``` + +In the above example, the tool will migrate the following items from the snapshot per the allowlist: +* The indices `Index1`, `.my_system_index`, and `logs-2023` +* The index template `logs_template` +* The component templates `component2` and `component7` \ No newline at end of file diff --git a/MetadataMigration/README.md b/MetadataMigration/README.md index d8e7ecdc0..1d4e4b61f 100644 --- a/MetadataMigration/README.md +++ b/MetadataMigration/README.md @@ -1,50 +1,153 @@ -# metadata-migration -## What is this tool? +## Metadata Migration +When performing a migration of a search cluster, the metadata items such as indexes, templates, configuration and processes need to be in place before document data can be moved. The metadata migration tool provides insight into what can be moved, if there are any issues moving data to the target cluster, and can deploy those changes. By inspecting and analyzing the metadata, issues can be discovered early in the overall migration timeline. -This tool exposes the underlying Reindex-From-Snapshot (RFS) core library in a executable that will migrate the templates and indices in a specified snapshot of a source cluster to a target cluster. In brief, it parses the contents of the snapshot to extract the settings/configuration of the templates and indices in the snapshot and then migrates those to the target cluster. The snapshot can either be local on disk or in S3. The user can apply allowlists to filter which templates/indices are migrated. If a template or index of the same name already exists on the target cluster, this tool will not overwrite the existing one on the target. +- [Metadata Migration](#metadata-migration) +- [Run Metadata Migration](#run-metadata-migration) + - [Metadata verification with evaluate command](#metadata-verification-with-evaluate-command) + - [Metadata migration with migrate command](#metadata-migration-with-migrate-command) + - [Metadata verification process](#metadata-verification-process) +- [How the tool works](#how-the-tool-works) + - [Breaking change compatibility](#breaking-change-compatibility) + - [Deprecation of Mapping Types](#deprecation-of-mapping-types) -The tool will also apply some basic transformations to the template and index settings in an attempt to handle upgrades between version-specific behavior. Further work is planned to flesh out this process; see [this design doc](./docs/DESIGN.md). +## Run Metadata Migration -## How to use the tool +Metadata migration is part of the Migration Assistant and can be accessed through the migration console. This is the recommended way to run this tool. If you are feeling adventurous, the tool can be run locally by following the instructions in the [DEVELOPER_GUIDE.md](./DEVELOPER_GUIDE.md). -You can kick off the locally tool using Gradle. +Metadata migration is a relatively fast process to execute so we recommend attempting this workflow as quickly as possible to discover any issues which could impact longer running migration steps. -### S3 Snapshot +### Metadata verification with evaluate command -From the root directory of the repo, run a CLI command like so: +By scanning the contents of the source cluster, applying filtering, and applying modifications a list of all items that will be migrated will be created. Any items not seen in this output will not be migrated onto the target cluster if the migrate command was to be run. This is a safety check before making modifications on the target cluster. ```shell -./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200' +console metadata evaluate ``` -In order for this succeed, you'll need to make sure you have valid AWS Credentials in your key ring (~/.aws/credentials) with permission to operate on the S3 URI specified. +
+ +Example evaluate command output + -### On-Disk Snapshot +``` +Starting Metadata Evaluation +Clusters: + Source: + Remote Cluster: OpenSearch 1.3.16 ConnectionContext(uri=http://localhost:33039, protocol=HTTP, insecure=false, compressionSupported=false) -From the root directory of the repo, run a CLI command like so: + Target: + Remote Cluster: OpenSearch 2.14.0 ConnectionContext(uri=http://localhost:33037, protocol=HTTP, insecure=false, compressionSupported=false) -```shell -./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --file-system-repo-path /snapshot --s3-region us-fake-1 --target-host http://hostname:9200' + +Migration Candidates: + Index Templates: + simple_index_template + + Component Templates: + simple_component_template + + Indexes: + blog_2023, movies_2023 + + Aliases: + alias1, movies-alias + + +Results: + 0 issue(s) detected ``` +
-### Handling Auth +### Metadata migration with migrate command -If your target cluster has basic auth enabled on it, you can supply those credentials to the tool via the CLI: +Running through the same data as the evaluate command all of the migrated items will be applied onto the target cluster. If re-run multiple times items that were previously migrated will not be recreated. If any items do need to be remigrated, please delete them from the target cluster and then rerun the evaluate then migrate commands to ensure the desired changes are made. ```shell -./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200 --target-username --target-password ' +console metadata migrate ``` -### Allowlisting the templates and indices to migrate +
+ +Example migrate command output + -By default, the tool has an empty allowlist for templates, meaning none will be migrated. In contrast, the default allowlist for indices is open, meaning all non-system indices (those not prefixed with `.`) will be migrated. You can tweak these allowlists with a comma-separated list of items you specifically with to migrate. If you specify an custom allowlist for the templates or indices, the default allowlist is disregarded and **only** the items you have in your allowlist will be moved. +``` +Starting Metadata Migration -```shell -./gradlew MetadataMigration:run --args='--snapshot-name reindex-from-snapshot --s3-local-dir /tmp/s3_files --s3-repo-uri s3://your-s3-uri --s3-region us-fake-1 --target-host http://hostname:9200 --index-allowlist Index1,.my_system_index,logs-2023 --index-template-allowlist logs_template --component-template-allowlist component2,component7' +Clusters: + Source: + Snapshot: OpenSearch 1.3.16 FileSystemRepo(repoRootDir=/tmp/junit10626813752669559861) + + Target: + Remote Cluster: OpenSearch 2.14.0 ConnectionContext(uri=http://localhost:33042, protocol=HTTP, insecure=false, compressionSupported=false) + + +Migrated Items: + Index Templates: + simple_index_template + + Component Templates: + simple_component_template + + Indexes: + blog_2023, movies_2023 + + Aliases: + alias1, movies-alias + + +Results: + 0 issue(s) detected +``` +
+ +### Metadata verification process + +Before moving on to additional migration steps, it is recommended to confirm details of your cluster. Depending on your configuration, this could be checking the sharding strategy or making sure index mappings are correctly defined by ingesting a test document. + +## How the tool works + +This tool gathers information from a source cluster, through a snapshot or through HTTP requests against the source cluster. These snapshots are fully compatible with Reindex-From-Snapshot (RFS) scenarios, [learn more](../DocumentsFromSnapshotMigration/README.md). + +After collecting information on the source cluster comparisons are made on the target cluster. If running a migration, any metadata items do not already exist will be created on the target cluster. + +### Breaking change compatibility + +Metadata migration needs to modify data from the source to the target versions to recreate items. Sometimes these features are no longer supported and have been removed from the target version. Sometimes these features are not available on the target version, which is especially true when downgrading. While this tool is meant to make this process easier, it is not exhaustive in its support. When encountering a compatibility issue or an important feature gap for your migration, please [search the issues](https://github.com/opensearch-project/opensearch-migrations/issues) and comment + upvote or a [create a new](https://github.com/opensearch-project/opensearch-migrations/issues/new/choose) issue if one cannot be found. + +#### Deprecation of Mapping Types +In Elasticsearch 6.8 the mapping types feature was discontinued in Elasticsearch 7.0+ which has created complexity in migrating to newer versions of Elasticsearch and OpenSearch, [learn more](https://www.elastic.co/guide/en/elasticsearch/reference/7.17/removal-of-types.html). + +As Metadata migration supports migrating from ES 6.8 on to the latest versions of OpenSearch this scenario is handled by removing the type mapping types and restructuring the template or index properties. Note that, at the time of this writing multiple type mappings are not supported, [tracking task](https://opensearch.atlassian.net/browse/MIGRATIONS-1778). + + +**Example starting state with mapping type foo (ES 6):** +```json +{ + "mappings": [ + { + "foo": { + "properties": { + "field1": { "type": "text" }, + "field2": { "type": "keyword" } + } + } + } + ] +} +``` + +**Example ending state with foo removed (ES 7):** +```json +{ + "mappings": { + "properties": { + "field1": { "type": "text" }, + "field2": { "type": "keyword" }, + } + } +} ``` -In the above example, the tool will migrate the following items from the snapshot per the allowlist: -* The indices `Index1`, `.my_system_index`, and `logs-2023` -* The index template `logs_template` -* The component templates `component2` and `component7` \ No newline at end of file +*Technical details are available, [view source code](../transformation/src/main/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemoval.java).* \ No newline at end of file diff --git a/MetadataMigration/build.gradle b/MetadataMigration/build.gradle index 1169bcfe6..7a80f19d3 100644 --- a/MetadataMigration/build.gradle +++ b/MetadataMigration/build.gradle @@ -14,11 +14,12 @@ dependencies { implementation project(":RFS") implementation project(':transformation') - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'org.slf4j', name: 'slf4j-api' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' - testImplementation testFixtures(project(path: ':RFS')) + testImplementation testFixtures(project(':RFS')) + testImplementation testFixtures(project(':testHelperFixtures')) testImplementation group: 'org.apache.logging.log4j', name: 'log4j-core' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' testImplementation group: 'org.junit.jupiter', name: 'junit-jupiter-api' diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataArgs.java b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataArgs.java index 30eb09199..9b05d988e 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataArgs.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataArgs.java @@ -1,54 +1,12 @@ package org.opensearch.migrations; - - import com.beust.jcommander.Parameter; -import com.beust.jcommander.ParametersDelegate; -import com.rfs.common.http.ConnectionContext; -import com.rfs.models.DataFilterArgs; public class MetadataArgs { @Parameter(names = {"--help", "-h"}, help = true, description = "Displays information about how to use this tool") public boolean help; - @Parameter(names = { "--snapshot-name" }, description = "The name of the snapshot to migrate") - public String snapshotName; - - @Parameter(names = { - "--file-system-repo-path" }, description = "The full path to the snapshot repo on the file system.") - public String fileSystemRepoPath; - - @Parameter(names = { - "--s3-local-dir" }, description = "The absolute path to the directory on local disk to download S3 files to") - public String s3LocalDirPath; - - @Parameter(names = { - "--s3-repo-uri" }, description = "The S3 URI of the snapshot repo, like: s3://my-bucket/dir1/dir2") - public String s3RepoUri; - - @Parameter(names = { - "--s3-region" }, description = "The AWS Region the S3 bucket is in, like: us-east-2") - public String s3Region; - - @ParametersDelegate - public ConnectionContext.SourceArgs sourceArgs = new ConnectionContext.SourceArgs(); - - @ParametersDelegate - public ConnectionContext.TargetArgs targetArgs = new ConnectionContext.TargetArgs(); - - @ParametersDelegate - public DataFilterArgs dataFilterArgs = new DataFilterArgs(); - - // https://opensearch.org/docs/2.11/api-reference/cluster-api/cluster-awareness/ - @Parameter(names = {"--min-replicas" }, description = "Optional. The minimum number of replicas configured for migrated indices on the target." - + " This can be useful for migrating to targets which use zonal deployments and require additional replicas to meet zone requirements. Default: 0") - public int minNumberOfReplicas = 0; - - @Parameter(required = false, names = { - "--otel-collector-endpoint" }, arity = 1, description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be" + @Parameter(names = { "--otel-collector-endpoint" }, description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be" + "forwarded. If no value is provided, metrics will not be forwarded.") - String otelCollectorEndpoint; - - @Parameter(names = {"--source-version" }, description = "Version of the source cluster, for example: Elasticsearch 7.10 or OS 1.3. Defaults to: ES_7.10", converter = VersionConverter.class) - public Version sourceVersion = Version.fromString("ES 7.10"); + public String otelCollectorEndpoint; } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataCommands.java b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataCommands.java new file mode 100644 index 000000000..20236ee0d --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataCommands.java @@ -0,0 +1,19 @@ +package org.opensearch.migrations; + +/** The list of supported commands for the metadata tool */ +public enum MetadataCommands { + /** Migrates items from a source and recreates them on the target cluster */ + Migrate, + + /** Inspects items from a source to determine which can be placed on a target cluster */ + Evaluate; + + public static MetadataCommands fromString(String s) { + for (var command : values()) { + if (command.name().toLowerCase().equals(s.toLowerCase())) { + return command; + } + } + throw new IllegalArgumentException("Unable to find matching command for text:" + s); + } +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataMigration.java b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataMigration.java index 54cb38cc5..7823e1e04 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataMigration.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/MetadataMigration.java @@ -1,8 +1,13 @@ package org.opensearch.migrations; +import java.util.Optional; + import org.opensearch.migrations.commands.Configure; import org.opensearch.migrations.commands.Evaluate; +import org.opensearch.migrations.commands.EvaluateArgs; import org.opensearch.migrations.commands.Migrate; +import org.opensearch.migrations.commands.MigrateArgs; +import org.opensearch.migrations.commands.Result; import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; import org.opensearch.migrations.tracing.ActiveContextTracker; import org.opensearch.migrations.tracing.ActiveContextTrackerByActivityType; @@ -17,46 +22,97 @@ public class MetadataMigration { public static void main(String[] args) throws Exception { - var arguments = new MetadataArgs(); - var jCommander = JCommander.newBuilder().addObject(arguments).build(); + var metadataArgs = new MetadataArgs(); + var migrateArgs = new MigrateArgs(); + var evaluateArgs = new EvaluateArgs(); + var jCommander = JCommander.newBuilder() + .addObject(metadataArgs) + .addCommand(migrateArgs) + .addCommand(evaluateArgs) + .build(); jCommander.parse(args); - if (arguments.help) { - jCommander.usage(); - return; - } - var context = new RootMetadataMigrationContext( - RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(arguments.otelCollectorEndpoint, "metadata", + RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(metadataArgs.otelCollectorEndpoint, "metadata", ProcessHelpers.getNodeInstanceName()), new CompositeContextTracker(new ActiveContextTracker(), new ActiveContextTrackerByActivityType()) ); - var meta = new MetadataMigration(arguments); + var meta = new MetadataMigration(); - log.info("Starting Metadata Migration"); + log.atInfo().setMessage("Command line arguments: {}\n").addArgument(String.join(" ", args)).log(); - var result = meta.migrate().execute(context); + if (metadataArgs.help || jCommander.getParsedCommand() == null) { + printTopLevelHelp(jCommander); + return; + } - log.info(result.toString()); - System.exit(result.getExitCode()); - } + if (migrateArgs.help || evaluateArgs.help) { + printCommandUsage(jCommander); + return; + } - private final MetadataArgs arguments; + var command = Optional.ofNullable(jCommander.getParsedCommand()) + .map(MetadataCommands::fromString) + .orElse(MetadataCommands.Migrate); + Result result; + switch (command) { + default: + case Migrate: + if (migrateArgs.help) { + printCommandUsage(jCommander); + return; + } - public MetadataMigration(MetadataArgs arguments) { - this.arguments = arguments; + log.info("Starting Metadata Migration"); + result = meta.migrate(migrateArgs).execute(context); + break; + case Evaluate: + if (evaluateArgs.help) { + printCommandUsage(jCommander); + return; + } + + log.info("Starting Metadata Evaluation"); + result = meta.evaluate(evaluateArgs).execute(context); + break; + } + log.atInfo().setMessage("{}").addArgument(result::asCliOutput).log(); + System.exit(result.getExitCode()); } public Configure configure() { return new Configure(); } - public Evaluate evaluate() { - return new Evaluate(); + public Evaluate evaluate(MigrateOrEvaluateArgs arguments) { + return new Evaluate(arguments); } - public Migrate migrate() { + public Migrate migrate(MigrateOrEvaluateArgs arguments) { return new Migrate(arguments); } + + private static void printTopLevelHelp(JCommander commander) { + var sb = new StringBuilder(); + sb.append("Usage: [options] [command] [commandOptions]"); + sb.append("Options:"); + for (var parameter : commander.getParameters()) { + sb.append(" " + parameter.getNames()); + sb.append(" " + parameter.getDescription()); + } + + sb.append("Commands:"); + for (var command : commander.getCommands().entrySet()) { + sb.append(" " + command.getKey()); + } + sb.append("\nUse --help with a specific command for more information."); + log.info(sb.toString()); + } + + private static void printCommandUsage(JCommander jCommander) { + var sb = new StringBuilder(); + jCommander.getUsageFormatter().usage(jCommander.getParsedCommand(), sb); + log.info(sb.toString()); + } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/MigrateOrEvaluateArgs.java b/MetadataMigration/src/main/java/org/opensearch/migrations/MigrateOrEvaluateArgs.java new file mode 100644 index 000000000..b05ffc5e2 --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/MigrateOrEvaluateArgs.java @@ -0,0 +1,54 @@ +package org.opensearch.migrations; + + + +import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; +import com.rfs.common.http.ConnectionContext; +import com.rfs.models.DataFilterArgs; + +public class MigrateOrEvaluateArgs { + @Parameter(names = {"--help", "-h"}, help = true, description = "Displays information about how to use this tool") + public boolean help; + + @Parameter(names = { "--snapshot-name" }, description = "The name of the snapshot to migrate") + public String snapshotName; + + @Parameter(names = { + "--file-system-repo-path" }, description = "The full path to the snapshot repo on the file system.") + public String fileSystemRepoPath; + + @Parameter(names = { + "--s3-local-dir" }, description = "The absolute path to the directory on local disk to download S3 files to") + public String s3LocalDirPath; + + @Parameter(names = { + "--s3-repo-uri" }, description = "The S3 URI of the snapshot repo, like: s3://my-bucket/dir1/dir2") + public String s3RepoUri; + + @Parameter(names = { + "--s3-region" }, description = "The AWS Region the S3 bucket is in, like: us-east-2") + public String s3Region; + + @ParametersDelegate + public ConnectionContext.SourceArgs sourceArgs = new ConnectionContext.SourceArgs(); + + @ParametersDelegate + public ConnectionContext.TargetArgs targetArgs = new ConnectionContext.TargetArgs(); + + @ParametersDelegate + public DataFilterArgs dataFilterArgs = new DataFilterArgs(); + + // https://opensearch.org/docs/2.11/api-reference/cluster-api/cluster-awareness/ + @Parameter(names = {"--min-replicas" }, description = "Optional. The minimum number of replicas configured for migrated indices on the target." + + " This can be useful for migrating to targets which use zonal deployments and require additional replicas to meet zone requirements. Default: 0") + public int minNumberOfReplicas = 0; + + @Parameter(required = false, names = { + "--otel-collector-endpoint" }, arity = 1, description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be" + + "forwarded. If no value is provided, metrics will not be forwarded.") + String otelCollectorEndpoint; + + @Parameter(names = {"--source-version" }, description = "Version of the source cluster, for example: Elasticsearch 7.10 or OS 1.3. Defaults to: ES_7.10", converter = VersionConverter.class) + public Version sourceVersion = Version.fromString("ES 7.10"); +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/ClusterReaderExtractor.java b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/ClusterReaderExtractor.java index d5415a17d..00ef9a4ce 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/ClusterReaderExtractor.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/ClusterReaderExtractor.java @@ -2,7 +2,8 @@ import java.nio.file.Path; -import org.opensearch.migrations.MetadataArgs; +import org.opensearch.migrations.MigrateOrEvaluateArgs; +import org.opensearch.migrations.Version; import org.opensearch.migrations.cluster.ClusterProviderRegistry; import org.opensearch.migrations.cluster.ClusterReader; @@ -11,14 +12,15 @@ import com.rfs.common.S3Repo; import com.rfs.common.S3Uri; import com.rfs.common.SourceRepo; +import com.rfs.common.http.ConnectionContext; import lombok.AllArgsConstructor; @AllArgsConstructor public class ClusterReaderExtractor { - final MetadataArgs arguments; + private final MigrateOrEvaluateArgs arguments; - public ClusterReader extractClusterReader() { - if (arguments.fileSystemRepoPath != null && arguments.s3RepoUri != null && arguments.sourceArgs.host != null) { + public ClusterReader extractClusterReader() { + if (arguments.fileSystemRepoPath == null && arguments.s3RepoUri == null && arguments.sourceArgs.host == null) { throw new ParameterException("No details on the source cluster found, please supply a connection details or a snapshot"); } if ((arguments.s3RepoUri != null) && (arguments.s3Region == null || arguments.s3LocalDirPath == null)) { @@ -26,7 +28,7 @@ public ClusterReader extractClusterReader() { } if (arguments.sourceArgs != null && arguments.sourceArgs.host != null) { - return ClusterProviderRegistry.getRemoteReader(arguments.sourceArgs.toConnectionContext()); + return getRemoteReader(arguments.sourceArgs.toConnectionContext()); } SourceRepo repo = null; @@ -38,6 +40,14 @@ public ClusterReader extractClusterReader() { throw new ParameterException("Unable to find valid resource provider"); } - return ClusterProviderRegistry.getSnapshotReader(arguments.sourceVersion, repo); + return getSnapshotReader(arguments.sourceVersion, repo); + } + + ClusterReader getRemoteReader(ConnectionContext connection) { + return ClusterProviderRegistry.getRemoteReader(connection); + } + + ClusterReader getSnapshotReader(Version sourceVersion, SourceRepo repo) { + return ClusterProviderRegistry.getSnapshotReader(sourceVersion, repo); } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Clusters.java b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Clusters.java index b967c0775..d29812cd6 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Clusters.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Clusters.java @@ -14,17 +14,17 @@ public class Clusters { private ClusterReader source; private ClusterWriter target; - public String toString() { + public String asCliOutput() { var sb = new StringBuilder(); sb.append("Clusters:" + System.lineSeparator()); if (getSource() != null) { - sb.append(" Source:" + System.lineSeparator()); - sb.append(" " + getSource() + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Source:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) + getSource() + System.lineSeparator()); sb.append(System.lineSeparator()); } if (getTarget() != null) { - sb.append(" Target:" + System.lineSeparator()); - sb.append(" " + getTarget() + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Target:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) + getTarget() + System.lineSeparator()); sb.append(System.lineSeparator()); } return sb.toString(); diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Format.java b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Format.java new file mode 100644 index 000000000..b4eb989a6 --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Format.java @@ -0,0 +1,15 @@ +package org.opensearch.migrations.cli; + +import lombok.experimental.UtilityClass; + +/** Shared formatting for command line interface components */ +@UtilityClass +public class Format { + + private static final String INDENT = " "; + + /** Indents to a given level for printing to the console */ + public static String indentToLevel(final int level) { + return INDENT.repeat(level); + } +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Items.java b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Items.java index fd35fdcdc..b2a12c459 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Items.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/cli/Items.java @@ -5,6 +5,7 @@ import lombok.Builder; import lombok.Data; +import lombok.NonNull; /** * Either items that are candidates for migration or have been migrated; @@ -12,34 +13,43 @@ @Builder @Data public class Items { - public boolean dryRun; - public List indexTemplates; - public List componentTemplates; - public List indexes; + static final String NONE_FOUND_MARKER = ""; + private final boolean dryRun; + @NonNull + private final List indexTemplates; + @NonNull + private final List componentTemplates; + @NonNull + private final List indexes; + @NonNull + private final List aliases; - public String toString() { + public String asCliOutput() { var sb = new StringBuilder(); if (isDryRun()) { sb.append("Migration Candidates:" + System.lineSeparator()); } else { sb.append("Migrated Items:" + System.lineSeparator()); } - sb.append(" Index Templates:" + System.lineSeparator()); - sb.append(" " + getPrintableList(getIndexTemplates()) + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Index Templates:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) + getPrintableList(getIndexTemplates()) + System.lineSeparator()); sb.append(System.lineSeparator()); - sb.append(" Component Templates:" + System.lineSeparator()); - sb.append(" " + getPrintableList(getComponentTemplates()) + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Component Templates:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) +getPrintableList(getComponentTemplates()) + System.lineSeparator()); sb.append(System.lineSeparator()); - sb.append(" Indexes:" + System.lineSeparator()); - sb.append(" " + getPrintableList(getIndexes()) + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Indexes:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) + getPrintableList(getIndexes()) + System.lineSeparator()); + sb.append(System.lineSeparator()); + sb.append(Format.indentToLevel(1) + "Aliases:" + System.lineSeparator()); + sb.append(Format.indentToLevel(2) +getPrintableList(getAliases()) + System.lineSeparator()); sb.append(System.lineSeparator()); return sb.toString(); } private String getPrintableList(List list) { if (list == null || list.isEmpty()) { - return ""; + return NONE_FOUND_MARKER; } - return list.stream().collect(Collectors.joining(", ")); + return list.stream().sorted().collect(Collectors.joining(", ")); } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Configure.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Configure.java index 9aef0d9d9..b958c864e 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Configure.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Configure.java @@ -6,7 +6,8 @@ public class Configure { public ConfigureResult execute() { - log.atError().setMessage("configure is not supported").log(); - return new ConfigureResult(9999); + var message = "configure is not supported"; + log.atError().setMessage(message).log(); + return new ConfigureResult(9999, message); } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/ConfigureResult.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/ConfigureResult.java index dac296884..294f4bb9a 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/ConfigureResult.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/ConfigureResult.java @@ -2,9 +2,18 @@ import lombok.AllArgsConstructor; import lombok.Getter; +import lombok.ToString; @AllArgsConstructor +@ToString public class ConfigureResult implements Result { @Getter private final int exitCode; + + @Getter + private final String errorMessage; + + public String asCliOutput() { + return this.toString(); + } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Evaluate.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Evaluate.java index 48d7b1d01..c7661b349 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Evaluate.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Evaluate.java @@ -1,12 +1,47 @@ package org.opensearch.migrations.commands; +import org.opensearch.migrations.MigrateOrEvaluateArgs; +import org.opensearch.migrations.MigrationMode; +import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; + +import com.beust.jcommander.ParameterException; import lombok.extern.slf4j.Slf4j; @Slf4j -public class Evaluate { +public class Evaluate extends MigratorEvaluatorBase { + + public Evaluate(MigrateOrEvaluateArgs arguments) { + super(arguments); + } + + public EvaluateResult execute(RootMetadataMigrationContext context) { + var migrationMode = MigrationMode.SIMULATE; + var evaluateResult = EvaluateResult.builder(); + + try { + log.info("Running Metadata Evaluation"); + + var clusters = createClusters(); + evaluateResult.clusters(clusters); + + var transformer = selectTransformer(clusters); + + var items = migrateAllItems(migrationMode, clusters, transformer, context); + evaluateResult.items(items); + } catch (ParameterException pe) { + log.atError().setMessage("Invalid parameter").setCause(pe).log(); + evaluateResult + .exitCode(INVALID_PARAMETER_CODE) + .errorMessage("Invalid parameter: " + pe.getMessage()) + .build(); + } catch (Throwable e) { + log.atError().setMessage("Unexpected failure").setCause(e).log(); + evaluateResult + .exitCode(UNEXPECTED_FAILURE_CODE) + .errorMessage("Unexpected failure: " + e.getMessage()) + .build(); + } - public EvaluateResult execute() { - log.atError().setMessage("evaluate is not supported").log(); - return new EvaluateResult(9999); + return evaluateResult.build(); } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateArgs.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateArgs.java new file mode 100644 index 000000000..23393a615 --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateArgs.java @@ -0,0 +1,9 @@ +package org.opensearch.migrations.commands; + +import org.opensearch.migrations.MigrateOrEvaluateArgs; + +import com.beust.jcommander.Parameters; + +@Parameters(commandNames = "evaluate", commandDescription = "Inspects items from a source to determine which can be placed on a target cluster") +public class EvaluateArgs extends MigrateOrEvaluateArgs { +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateResult.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateResult.java index e8dc9822f..8f4273e06 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateResult.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/EvaluateResult.java @@ -1,10 +1,16 @@ package org.opensearch.migrations.commands; -import lombok.AllArgsConstructor; +import org.opensearch.migrations.cli.Clusters; +import org.opensearch.migrations.cli.Items; + +import lombok.Builder; import lombok.Getter; -@AllArgsConstructor -public class EvaluateResult implements Result { - @Getter +@Builder +@Getter +public class EvaluateResult implements MigrationItemResult { + private final Clusters clusters; + private final Items items; + private final String errorMessage; private final int exitCode; } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Migrate.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Migrate.java index ad0667ac4..e09d0d76e 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Migrate.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Migrate.java @@ -1,82 +1,33 @@ package org.opensearch.migrations.commands; -import java.util.ArrayList; - -import org.opensearch.migrations.MetadataArgs; -import org.opensearch.migrations.cli.ClusterReaderExtractor; -import org.opensearch.migrations.cli.Clusters; -import org.opensearch.migrations.cli.Items; -import org.opensearch.migrations.cluster.ClusterProviderRegistry; +import org.opensearch.migrations.MigrateOrEvaluateArgs; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; import com.beust.jcommander.ParameterException; -import com.rfs.transformers.TransformFunctions; -import com.rfs.worker.IndexRunner; -import com.rfs.worker.MetadataRunner; import lombok.extern.slf4j.Slf4j; @Slf4j -public class Migrate { - - static final int INVALID_PARAMETER_CODE = 999; - static final int UNEXPECTED_FAILURE_CODE = 888; - private final MetadataArgs arguments; - private final ClusterReaderExtractor clusterReaderCliExtractor; +public class Migrate extends MigratorEvaluatorBase { - public Migrate(MetadataArgs arguments) { - this.arguments = arguments; - clusterReaderCliExtractor = new ClusterReaderExtractor(arguments); + public Migrate(MigrateOrEvaluateArgs arguments) { + super(arguments); } public MigrateResult execute(RootMetadataMigrationContext context) { + var migrationMode = MigrationMode.PERFORM; var migrateResult = MigrateResult.builder(); - log.atInfo().setMessage("Command line arguments {0}").addArgument(arguments::toString).log(); try { - log.info("Running Metadata worker"); - - var clusters = Clusters.builder(); - var sourceCluster = clusterReaderCliExtractor.extractClusterReader(); - clusters.source(sourceCluster); - - var targetCluster = ClusterProviderRegistry.getRemoteWriter(arguments.targetArgs.toConnectionContext(), arguments.dataFilterArgs); - clusters.target(targetCluster); - migrateResult.clusters(clusters.build()); - - var transformer = TransformFunctions.getTransformer( - sourceCluster.getVersion(), - targetCluster.getVersion(), - arguments.minNumberOfReplicas - ); - - log.info("Using transformation " + transformer.toString()); - - var metadataResults = new MetadataRunner( - arguments.snapshotName, - sourceCluster.getGlobalMetadata(), - targetCluster.getGlobalMetadataCreator(), - transformer - ).migrateMetadata(context.createMetadataMigrationContext()); + log.info("Running Metadata Migration"); - var items = Items.builder(); - var indexTemplates = new ArrayList(); - indexTemplates.addAll(metadataResults.getLegacyTemplates()); - indexTemplates.addAll(metadataResults.getIndexTemplates()); - items.indexTemplates(indexTemplates); - items.componentTemplates(metadataResults.getComponentTemplates()); + var clusters = createClusters(); + migrateResult.clusters(clusters); - log.info("Metadata copy complete."); + var transformer = selectTransformer(clusters); - var indexes = new IndexRunner( - arguments.snapshotName, - sourceCluster.getIndexMetadata(), - targetCluster.getIndexCreator(), - transformer, - arguments.dataFilterArgs.indexAllowlist - ).migrateIndices(context.createIndexContext()); - items.indexes(indexes); - migrateResult.items(items.build()); - log.info("Index copy complete."); + var items = migrateAllItems(migrationMode, clusters, transformer, context); + migrateResult.items(items); } catch (ParameterException pe) { log.atError().setMessage("Invalid parameter").setCause(pe).log(); migrateResult diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateArgs.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateArgs.java new file mode 100644 index 000000000..0e982da28 --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateArgs.java @@ -0,0 +1,9 @@ +package org.opensearch.migrations.commands; + +import org.opensearch.migrations.MigrateOrEvaluateArgs; + +import com.beust.jcommander.Parameters; + +@Parameters(commandNames = "migrate", commandDescription = "Migrates items from a source and recreates them on the target cluster") +public class MigrateArgs extends MigrateOrEvaluateArgs { +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateResult.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateResult.java index fa9e008eb..51b2ba0b2 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateResult.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrateResult.java @@ -1,7 +1,5 @@ package org.opensearch.migrations.commands; -import org.apache.logging.log4j.util.Strings; - import org.opensearch.migrations.cli.Clusters; import org.opensearch.migrations.cli.Items; @@ -10,28 +8,9 @@ @Builder @Getter -public class MigrateResult implements Result { +public class MigrateResult implements MigrationItemResult { private final Clusters clusters; private final Items items; private final String errorMessage; private final int exitCode; - - public String toString() { - var sb = new StringBuilder(); - if (getClusters() != null) { - sb.append(getClusters() + System.lineSeparator()); - } - if (getItems() != null) { - sb.append(getItems() + System.lineSeparator()); - } - sb.append("Results:" + System.lineSeparator()); - if (Strings.isNotBlank(getErrorMessage())) { - sb.append(" Issue(s) detected" + System.lineSeparator()); - sb.append("Issues:" + System.lineSeparator()); - sb.append(" " + getErrorMessage() + System.lineSeparator()); - } else { - sb.append(" " + getExitCode() + " issue(s) detected" + System.lineSeparator()); - } - return sb.toString(); - } } diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrationItemResult.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrationItemResult.java new file mode 100644 index 000000000..67396f11a --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigrationItemResult.java @@ -0,0 +1,32 @@ +package org.opensearch.migrations.commands; + +import org.apache.logging.log4j.util.Strings; + +import org.opensearch.migrations.cli.Clusters; +import org.opensearch.migrations.cli.Format; +import org.opensearch.migrations.cli.Items; + +/** All shared cli result information */ +public interface MigrationItemResult extends Result { + Clusters getClusters(); + Items getItems(); + + default String asCliOutput() { + var sb = new StringBuilder(); + if (getClusters() != null) { + sb.append(getClusters().asCliOutput() + System.lineSeparator()); + } + if (getItems() != null) { + sb.append(getItems().asCliOutput() + System.lineSeparator()); + } + sb.append("Results:" + System.lineSeparator()); + if (Strings.isNotBlank(getErrorMessage())) { + sb.append(Format.indentToLevel(1) + "Issue(s) detected" + System.lineSeparator()); + sb.append("Issues:" + System.lineSeparator()); + sb.append(Format.indentToLevel(1) + getErrorMessage() + System.lineSeparator()); + } else { + sb.append(Format.indentToLevel(1) + getExitCode() + " issue(s) detected" + System.lineSeparator()); + } + return sb.toString(); + } +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigratorEvaluatorBase.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigratorEvaluatorBase.java new file mode 100644 index 000000000..859d5addc --- /dev/null +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/MigratorEvaluatorBase.java @@ -0,0 +1,98 @@ +package org.opensearch.migrations.commands; + +import java.util.ArrayList; + +import org.opensearch.migrations.MigrateOrEvaluateArgs; +import org.opensearch.migrations.MigrationMode; +import org.opensearch.migrations.cli.ClusterReaderExtractor; +import org.opensearch.migrations.cli.Clusters; +import org.opensearch.migrations.cli.Items; +import org.opensearch.migrations.cluster.ClusterProviderRegistry; +import org.opensearch.migrations.metadata.GlobalMetadataCreatorResults; +import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; + +import com.rfs.transformers.TransformFunctions; +import com.rfs.transformers.Transformer; +import com.rfs.worker.IndexMetadataResults; +import com.rfs.worker.IndexRunner; +import com.rfs.worker.MetadataRunner; +import lombok.extern.slf4j.Slf4j; + +/** Shared functionality between migration and evaluation commands */ +@Slf4j +public abstract class MigratorEvaluatorBase { + + static final int INVALID_PARAMETER_CODE = 999; + static final int UNEXPECTED_FAILURE_CODE = 888; + + protected final MigrateOrEvaluateArgs arguments; + protected final ClusterReaderExtractor clusterReaderCliExtractor; + + protected MigratorEvaluatorBase(MigrateOrEvaluateArgs arguments) { + this.arguments = arguments; + this.clusterReaderCliExtractor = new ClusterReaderExtractor(arguments); + } + + protected Clusters createClusters() { + var clusters = Clusters.builder(); + var sourceCluster = clusterReaderCliExtractor.extractClusterReader(); + clusters.source(sourceCluster); + + var targetCluster = ClusterProviderRegistry.getRemoteWriter(arguments.targetArgs.toConnectionContext(), arguments.dataFilterArgs); + clusters.target(targetCluster); + return clusters.build(); + } + + protected Transformer selectTransformer(Clusters clusters) { + var transformer = TransformFunctions.getTransformer( + clusters.getSource().getVersion(), + clusters.getTarget().getVersion(), + arguments.minNumberOfReplicas + ); + log.info("Selected transformer " + transformer.toString()); + return transformer; + } + + protected Items migrateAllItems(MigrationMode migrationMode, Clusters clusters, Transformer transformer, RootMetadataMigrationContext context) { + var items = Items.builder(); + items.dryRun(migrationMode.equals(MigrationMode.SIMULATE)); + var metadataResults = migrateGlobalMetadata(migrationMode, clusters, transformer, context); + + var indexTemplates = new ArrayList(); + indexTemplates.addAll(metadataResults.getLegacyTemplates()); + indexTemplates.addAll(metadataResults.getIndexTemplates()); + items.indexTemplates(indexTemplates); + items.componentTemplates(metadataResults.getComponentTemplates()); + + var indexResults = migrateIndices(migrationMode, clusters, transformer, context); + items.indexes(indexResults.getIndexNames()); + items.aliases(indexResults.getAliases()); + + return items.build(); + } + + private GlobalMetadataCreatorResults migrateGlobalMetadata(MigrationMode mode, Clusters clusters, Transformer transformer, RootMetadataMigrationContext context) { + var metadataRunner = new MetadataRunner( + arguments.snapshotName, + clusters.getSource().getGlobalMetadata(), + clusters.getTarget().getGlobalMetadataCreator(), + transformer + ); + var metadataResults = metadataRunner.migrateMetadata(mode, context.createMetadataMigrationContext()); + log.info("Metadata copy complete."); + return metadataResults; + } + + private IndexMetadataResults migrateIndices(MigrationMode mode, Clusters clusters, Transformer transformer, RootMetadataMigrationContext context) { + var indexRunner = new IndexRunner( + arguments.snapshotName, + clusters.getSource().getIndexMetadata(), + clusters.getTarget().getIndexCreator(), + transformer, + arguments.dataFilterArgs.indexAllowlist + ); + var indexResults = indexRunner.migrateIndices(mode, context.createIndexContext()); + log.info("Index copy complete."); + return indexResults; + } +} diff --git a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Result.java b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Result.java index 552d706fb..95cc78016 100644 --- a/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Result.java +++ b/MetadataMigration/src/main/java/org/opensearch/migrations/commands/Result.java @@ -3,4 +3,7 @@ /** All shared cli result information */ public interface Result { int getExitCode(); + String getErrorMessage(); + /** Render this result as a string for displaying on the command line */ + String asCliOutput(); } diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/EndToEndTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/EndToEndTest.java index 9fd72c870..f438aef2c 100644 --- a/MetadataMigration/src/test/java/org/opensearch/migrations/EndToEndTest.java +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/EndToEndTest.java @@ -3,13 +3,15 @@ import java.io.File; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.stream.Stream; -import org.hamcrest.Matchers; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.opensearch.migrations.commands.MigrationItemResult; import org.opensearch.migrations.metadata.tracing.MetadataMigrationTestContext; import org.opensearch.migrations.snapshot.creation.tracing.SnapshotTestContext; @@ -23,60 +25,72 @@ import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; +import static org.hamcrest.CoreMatchers.allOf; +import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; /** * Tests focused on setting up whole source clusters, performing a migration, and validation on the target cluster */ -@Tag("longTest") +@Tag("isolatedTest") @Slf4j class EndToEndTest { @TempDir private File localDirectory; - @ParameterizedTest(name = "Medium of transfer {0}") - @EnumSource(TransferMedium.class) - void metadataMigrateFrom_ES_v6_8(TransferMedium medium) throws Exception { + private static Stream scenarios() { + return Stream.of( + Arguments.of(TransferMedium.Http, MetadataCommands.Evaluate), + Arguments.of(TransferMedium.SnapshotImage, MetadataCommands.Migrate), + Arguments.of(TransferMedium.Http, MetadataCommands.Migrate) + ); + } + + @ParameterizedTest(name = "Command {1}, Medium of transfer {0}") + @MethodSource(value = "scenarios") + void metadataMigrateFrom_ES_v6_8(TransferMedium medium, MetadataCommands command) throws Exception { try ( final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V6_8_23); final var targetCluster = new SearchClusterContainer(SearchClusterContainer.OS_V2_14_0) ) { - migrateFrom_ES(sourceCluster, targetCluster, medium); + migrateFrom_ES(sourceCluster, targetCluster, medium, command); } } - @ParameterizedTest(name = "Medium of transfer {0}") - @EnumSource(TransferMedium.class) - void metadataMigrateFrom_ES_v7_17(TransferMedium medium) throws Exception { + @ParameterizedTest(name = "Command {1}, Medium of transfer {0}") + @MethodSource(value = "scenarios") + void metadataMigrateFrom_ES_v7_17(TransferMedium medium, MetadataCommands command) throws Exception { try ( final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V7_17); final var targetCluster = new SearchClusterContainer(SearchClusterContainer.OS_V2_14_0) ) { - migrateFrom_ES(sourceCluster, targetCluster, medium); + migrateFrom_ES(sourceCluster, targetCluster, medium, command); } } - @ParameterizedTest(name = "Medium of transfer {0}") - @EnumSource(TransferMedium.class) - void metadataMigrateFrom_ES_v7_10(TransferMedium medium) throws Exception { + @ParameterizedTest(name = "Command {1}, Medium of transfer {0}") + @MethodSource(value = "scenarios") + void metadataMigrateFrom_ES_v7_10(TransferMedium medium, MetadataCommands command) throws Exception { try ( final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.ES_V7_10_2); final var targetCluster = new SearchClusterContainer(SearchClusterContainer.OS_V2_14_0) ) { - migrateFrom_ES(sourceCluster, targetCluster, medium); + migrateFrom_ES(sourceCluster, targetCluster, medium, command); } } - @ParameterizedTest(name = "Medium of transfer {0}") - @EnumSource(TransferMedium.class) - void metadataMigrateFrom_OS_v1_3(TransferMedium medium) throws Exception { + @ParameterizedTest(name = "Command {1}, Medium of transfer {0}") + @MethodSource(value = "scenarios") + void metadataMigrateFrom_OS_v1_3(TransferMedium medium, MetadataCommands command) throws Exception { try ( final var sourceCluster = new SearchClusterContainer(SearchClusterContainer.OS_V1_3_16); final var targetCluster = new SearchClusterContainer(SearchClusterContainer.OS_V2_14_0) ) { - migrateFrom_ES(sourceCluster, targetCluster, medium); + migrateFrom_ES(sourceCluster, targetCluster, medium, command); } } @@ -89,7 +103,8 @@ private enum TransferMedium { private void migrateFrom_ES( final SearchClusterContainer sourceCluster, final SearchClusterContainer targetCluster, - final TransferMedium medium + final TransferMedium medium, + final MetadataCommands command ) { // ACTION: Set up the source/target clusters var bothClustersStarted = CompletableFuture.allOf( @@ -106,23 +121,25 @@ private void migrateFrom_ES( throw new RuntimeException("This test cannot handle the source cluster version" + sourceVersion); } + var testData = new TestData(); // Create the component and index templates var sourceClusterOperations = new ClusterOperations(sourceCluster.getUrl()); - var compoTemplateName = "simple_component_template"; - var indexTemplateName = "simple_index_template"; if (sourceIsES7_X) { - sourceClusterOperations.createES7Templates(compoTemplateName, indexTemplateName, "author", "blog*"); + sourceClusterOperations.createES7Templates(testData.compoTemplateName, testData.indexTemplateName, "author", "blog*"); } else if (sourceIsES6_8) { - sourceClusterOperations.createES6LegacyTemplate(indexTemplateName, "movies*"); + sourceClusterOperations.createES6LegacyTemplate(testData.indexTemplateName, "blog*"); } // Creates a document that uses the template - var blogIndexName = "blog_2023"; - sourceClusterOperations.createDocument(blogIndexName, "222", "{\"author\":\"Tobias Funke\"}"); - var movieIndexName = "movies_2023"; - sourceClusterOperations.createDocument(movieIndexName,"123", "{\"title\":\"This is spinal tap\"}"); + sourceClusterOperations.createDocument(testData.blogIndexName, "222", "{\"author\":\"Tobias Funke\"}"); + sourceClusterOperations.createDocument(testData.movieIndexName,"123", "{\"title\":\"This is spinal tap\"}"); - var arguments = new MetadataArgs(); + sourceClusterOperations.createAlias(testData.aliasName, "movies*"); + + var aliasName = "movies-alias"; + sourceClusterOperations.createAlias(aliasName, "movies*"); + + var arguments = new MigrateOrEvaluateArgs(); switch (medium) { case SnapshotImage: @@ -155,35 +172,89 @@ private void migrateFrom_ES( arguments.targetArgs.host = targetCluster.getUrl(); var dataFilterArgs = new DataFilterArgs(); - dataFilterArgs.indexAllowlist = List.of(blogIndexName, movieIndexName); - dataFilterArgs.componentTemplateAllowlist = List.of(compoTemplateName); - dataFilterArgs.indexTemplateAllowlist = List.of(indexTemplateName); + dataFilterArgs.indexAllowlist = List.of(testData.blogIndexName, testData.movieIndexName); + dataFilterArgs.componentTemplateAllowlist = List.of(testData.compoTemplateName); + dataFilterArgs.indexTemplateAllowlist = List.of(testData.indexTemplateName); arguments.dataFilterArgs = dataFilterArgs; // ACTION: Migrate the templates var metadataContext = MetadataMigrationTestContext.factory().noOtelTracking(); - var result = new MetadataMigration(arguments).migrate().execute(metadataContext); + var metadata = new MetadataMigration(); + + MigrationItemResult result; + if (MetadataCommands.Migrate.equals(command)) { + result = metadata.migrate(arguments).execute(metadataContext); + } else { + result = metadata.evaluate(arguments).execute(metadataContext); + } + + verifyCommandResults(result, sourceIsES6_8, testData); + + verifyTargetCluster(targetCluster, command, sourceIsES6_8, testData); + } + + private static class TestData { + final String compoTemplateName = "simple_component_template"; + final String indexTemplateName = "simple_index_template"; + final String aliasInTemplate = "alias1"; + final String blogIndexName = "blog_2023"; + final String movieIndexName = "movies_2023"; + final String aliasName = "movies-alias"; + } + private void verifyCommandResults( + MigrationItemResult result, + boolean sourceIsES6_8, + TestData testData) { log.info(result.toString()); assertThat(result.getExitCode(), equalTo(0)); + var migratedItems = result.getItems(); + assertThat(migratedItems.getIndexTemplates(), containsInAnyOrder(testData.indexTemplateName)); + assertThat(migratedItems.getComponentTemplates(), equalTo(sourceIsES6_8 ? List.of() : List.of(testData.compoTemplateName))); + assertThat(migratedItems.getIndexes(), containsInAnyOrder(testData.blogIndexName, testData.movieIndexName)); + assertThat(migratedItems.getAliases(), containsInAnyOrder(testData.aliasInTemplate, testData.aliasName)); + } + + private void verifyTargetCluster( + SearchClusterContainer targetCluster, + MetadataCommands command, + boolean sourceIsES6_8, + TestData testData + ) { + var expectUpdatesOnTarget = MetadataCommands.Migrate.equals(command); + // If the command was migrate, the target cluster should have the items, if not they + var verifyResponseCode = expectUpdatesOnTarget ? equalTo(200) : equalTo(404); + // Check that the index was migrated var targetClusterOperations = new ClusterOperations(targetCluster.getUrl()); - var res = targetClusterOperations.get("/" + blogIndexName); - assertThat(res.getValue(), res.getKey(), equalTo(200)); + var res = targetClusterOperations.get("/" + testData.blogIndexName); + assertThat(res.getValue(), res.getKey(), verifyResponseCode); + + res = targetClusterOperations.get("/" + testData.movieIndexName); + assertThat(res.getValue(), res.getKey(), verifyResponseCode); - res = targetClusterOperations.get("/" + movieIndexName); + res = targetClusterOperations.get("/" + testData.aliasName); + assertThat(res.getValue(), res.getKey(), verifyResponseCode); + if (expectUpdatesOnTarget) { + assertThat(res.getValue(), containsString(testData.movieIndexName)); + } + + res = targetClusterOperations.get("/_aliases"); assertThat(res.getValue(), res.getKey(), equalTo(200)); - + var verifyAliasWasListed = allOf(containsString(testData.aliasInTemplate), containsString(testData.aliasName)); + assertThat(res.getValue(), expectUpdatesOnTarget ? verifyAliasWasListed : not(verifyAliasWasListed)); + // Check that the templates were migrated - if (sourceIsES7_X) { - res = targetClusterOperations.get("/_index_template/" + indexTemplateName); - assertThat(res.getValue(), res.getKey(), equalTo(200)); - assertThat(res.getValue(), Matchers.containsString("composed_of\":[\"" + compoTemplateName + "\"]")); - } else if (sourceIsES6_8) { - res = targetClusterOperations.get("/_template/" + indexTemplateName); - assertThat(res.getValue(), res.getKey(), equalTo(200)); + if (sourceIsES6_8) { + res = targetClusterOperations.get("/_template/" + testData.indexTemplateName); + assertThat(res.getValue(), res.getKey(), verifyResponseCode); + } else { + res = targetClusterOperations.get("/_index_template/" + testData.indexTemplateName); + assertThat(res.getValue(), res.getKey(), verifyResponseCode); + var verifyBodyHasComponentTemplate = containsString("composed_of\":[\"" + testData.compoTemplateName + "\"]"); + assertThat(res.getValue(), expectUpdatesOnTarget ? verifyBodyHasComponentTemplate : not(verifyBodyHasComponentTemplate)); } } } diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/MetadataMigrationTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/MetadataMigrationTest.java new file mode 100644 index 000000000..37b59b551 --- /dev/null +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/MetadataMigrationTest.java @@ -0,0 +1,53 @@ +package org.opensearch.migrations; + +import java.util.List; + +import org.junit.jupiter.api.Test; + +import org.opensearch.migrations.testutils.CloseableLogSetup; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; + +public class MetadataMigrationTest { + + @Test + void testMain_expectTopLevelHelp() throws Exception { + var testCases = List.of( + new String[]{}, + new String[]{"-h"}, + new String[]{"--help"} + ); + for (var testCase : testCases) { + try (var closeableLogSetup = new CloseableLogSetup(MetadataMigration.class.getName())) { + MetadataMigration.main(testCase); + + var logEvents = closeableLogSetup.getLogEvents(); + + assertThat(logEvents, hasSize(2)); + assertThat(logEvents.get(0), containsString("Command line arguments")); + assertThat(logEvents.get(1), containsString("Usage: [options] [command] [commandOptions]")); + } + } + } + + @Test + void testMain_expectCommandHelp() throws Exception { + var testCases = List.of( + new String[]{"evaluate", "-h"}, + new String[]{"migrate", "--help"} + ); + for (var testCase : testCases) { + try (var closeableLogSetup = new CloseableLogSetup(MetadataMigration.class.getName())) { + MetadataMigration.main(testCase); + + var logEvents = closeableLogSetup.getLogEvents(); + + assertThat(logEvents, hasSize(2)); + assertThat(logEvents.get(0), containsString("Command line arguments")); + assertThat(logEvents.get(1), containsString("Usage: " + testCase[0] + " [options]")); + } + } + } +} diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClusterReaderExtractorTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClusterReaderExtractorTest.java new file mode 100644 index 000000000..dd6325aa5 --- /dev/null +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClusterReaderExtractorTest.java @@ -0,0 +1,105 @@ +package org.opensearch.migrations.cli; + +import org.junit.jupiter.api.Test; + +import org.opensearch.migrations.MigrateOrEvaluateArgs; +import org.opensearch.migrations.Version; +import org.opensearch.migrations.cluster.ClusterReader; + +import com.beust.jcommander.ParameterException; +import com.rfs.common.FileSystemRepo; +import com.rfs.common.S3Repo; +import com.rfs.common.http.ConnectionContext; +import org.mockito.ArgumentCaptor; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; + + +public class ClusterReaderExtractorTest { + @Test + void testExtractClusterReader_noSnapshotOrRemote() { + var args = new MigrateOrEvaluateArgs(); + var extractor = new ClusterReaderExtractor(args); + + var exception = assertThrows(ParameterException.class, () -> extractor.extractClusterReader()); + assertThat(args.toString(), exception.getMessage(), equalTo("No details on the source cluster found, please supply a connection details or a snapshot")); + } + + @Test + void testExtractClusterReader_invalidS3Snapshot_missingRegion() { + var args = new MigrateOrEvaluateArgs(); + args.s3RepoUri = "foo.bar"; + args.s3LocalDirPath = "fizz.buzz"; + var extractor = new ClusterReaderExtractor(args); + + var exception = assertThrows(ParameterException.class, () -> extractor.extractClusterReader()); + assertThat(exception.getMessage(), equalTo("If an s3 repo is being used, s3-region and s3-local-dir-path must be set")); + } + + @Test + void testExtractClusterReader_invalidS3Snapshot_missingLocalDirPath() { + var args = new MigrateOrEvaluateArgs(); + args.s3RepoUri = "foo.bar"; + args.s3Region = "us-west-1"; + var extractor = new ClusterReaderExtractor(args); + + var exception = assertThrows(ParameterException.class, () -> extractor.extractClusterReader()); + assertThat(exception.getMessage(), equalTo("If an s3 repo is being used, s3-region and s3-local-dir-path must be set")); + } + + @Test + void testExtractClusterReader_validLocalSnapshot() { + var args = new MigrateOrEvaluateArgs(); + args.fileSystemRepoPath = "foo.bar"; + args.sourceVersion = Version.fromString("OS 1.1.1"); + var extractor = spy(new ClusterReaderExtractor(args)); + var mockReader = mock(ClusterReader.class); + doReturn(mockReader).when(extractor).getSnapshotReader(eq(args.sourceVersion), any(FileSystemRepo.class)); + + var result = extractor.extractClusterReader(); + assertThat(result, equalTo(mockReader)); + + verify(extractor).getSnapshotReader(eq(args.sourceVersion), any(FileSystemRepo.class)); + } + + @Test + void testExtractClusterReader_validS3Snapshot() { + var args = new MigrateOrEvaluateArgs(); + args.s3RepoUri = "foo.bar"; + args.s3Region = "us-west-1"; + args.s3LocalDirPath = "fizz.buzz"; + args.sourceVersion = Version.fromString("OS 9.9.9"); + var extractor = spy(new ClusterReaderExtractor(args)); + var mockReader = mock(ClusterReader.class); + doReturn(mockReader).when(extractor).getSnapshotReader(eq(args.sourceVersion), any(S3Repo.class)); + + var result = extractor.extractClusterReader(); + assertThat(result, equalTo(mockReader)); + + verify(extractor).getSnapshotReader(eq(args.sourceVersion), any(S3Repo.class)); + } + + @Test + void testExtractClusterReader_validRemote() { + var args = new MigrateOrEvaluateArgs(); + args.sourceArgs.host = "http://foo.bar"; + var extractor = spy(new ClusterReaderExtractor(args)); + var mockReader = mock(ClusterReader.class); + doReturn(mockReader).when(extractor).getRemoteReader(any()); + + var result = extractor.extractClusterReader(); + assertThat(result, equalTo(mockReader)); + + var foundContext = ArgumentCaptor.forClass(ConnectionContext.class); + verify(extractor).getRemoteReader(foundContext.capture()); + assertThat(args.sourceArgs.toConnectionContext(), equalTo(foundContext.getValue())); + } +} diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClustersTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClustersTest.java new file mode 100644 index 000000000..6a037bf48 --- /dev/null +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ClustersTest.java @@ -0,0 +1,55 @@ +package org.opensearch.migrations.cli; + +import org.junit.jupiter.api.Test; + +import org.opensearch.migrations.cluster.ClusterReader; +import org.opensearch.migrations.cluster.ClusterWriter; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.migrations.matchers.HasLineCount.hasLineCount; +import static org.mockito.Mockito.mock; + +public class ClustersTest { + @Test + void testAsString_empty() { + var clusters = Clusters.builder().build(); + + var result = clusters.asCliOutput(); + + assertThat(result, containsString("Clusters:")); + assertThat(result, not(containsString("Source:"))); + assertThat(result, not(containsString("Target:"))); + assertThat(result, hasLineCount(1)); + } + + @Test + void testAsString_withSource() { + var clusters = Clusters.builder() + .source(mock(ClusterReader.class)) + .build(); + + var result = clusters.asCliOutput(); + + assertThat(result, containsString("Clusters:")); + assertThat(result, containsString("Source:")); + assertThat(result, not(containsString("Target:"))); + assertThat(result, hasLineCount(3)); + } + + @Test + void testAsString_withSourceAndTarget() { + var clusters = Clusters.builder() + .source(mock(ClusterReader.class)) + .target(mock(ClusterWriter.class)) + .build(); + + var result = clusters.asCliOutput(); + + assertThat(result, containsString("Clusters:")); + assertThat(result, containsString("Source:")); + assertThat(result, containsString("Target:")); + assertThat(result, hasLineCount(6)); + } +} diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ItemsTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ItemsTest.java new file mode 100644 index 000000000..6a6f33930 --- /dev/null +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/cli/ItemsTest.java @@ -0,0 +1,77 @@ +package org.opensearch.migrations.cli; + +import java.util.List; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.migrations.matchers.ContainsStringCount.containsStringCount; +import static org.opensearch.migrations.matchers.HasLineCount.hasLineCount; + +public class ItemsTest { + @Test + void testAsString_empty() { + var items = Items.builder() + .indexTemplates(List.of()) + .componentTemplates(List.of()) + .indexes(List.of()) + .aliases(List.of()) + .build(); + + var result = items.asCliOutput(); + + assertThat(result, containsString("Migrated Items:")); + assertThat(result, containsString("Index Templates:")); + assertThat(result, containsString("Component Templates:")); + assertThat(result, containsString("Indexes:")); + assertThat(result, containsString("Aliases:")); + assertThat(result, containsStringCount(Items.NONE_FOUND_MARKER, 4)); + assertThat(result, hasLineCount(12)); + } + + @Test + void testAsString_full() { + var items = Items.builder() + .indexTemplates(List.of("it1", "it2")) + .componentTemplates(List.of("ct1", "ct2")) + .indexes(List.of("i1", "i2")) + .aliases(List.of("a1", "a2")) + .build(); + + var result = items.asCliOutput(); + + assertThat(result, containsString("Migrated Items:")); + assertThat(result, containsString("Index Templates:")); + assertThat(result, containsString("it1, it2")); + assertThat(result, containsString("Component Templates:")); + assertThat(result, containsString("ct1, ct2")); + assertThat(result, containsString("Indexes:")); + assertThat(result, containsString("i1, i2")); + assertThat(result, containsString("Aliases:")); + assertThat(result, containsString("a1, a2")); + assertThat(result, containsStringCount(Items.NONE_FOUND_MARKER, 0)); + assertThat(result, hasLineCount(12)); + } + + @Test + void testAsString_itemOrdering() { + var items = Items.builder() + .indexTemplates(List.of()) + .componentTemplates(List.of()) + .indexes(List.of("i1", "i2", "i5", "i3", "i4")) + .aliases(List.of()) + .build(); + + var result = items.asCliOutput(); + + assertThat(result, containsString("Migrated Items:")); + assertThat(result, containsString("Index Templates:")); + assertThat(result, containsString("i1, i2, i3, i4, i5")); + assertThat(result, containsString("Component Templates:")); + assertThat(result, containsString("Indexes:")); + assertThat(result, containsString("Aliases:")); + assertThat(result, containsStringCount(Items.NONE_FOUND_MARKER, 3)); + assertThat(result, hasLineCount(12)); + } +} diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/ConfigureTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/ConfigureTest.java index 4b59154e2..10fb99e10 100644 --- a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/ConfigureTest.java +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/ConfigureTest.java @@ -2,18 +2,16 @@ import org.junit.jupiter.api.Test; -import org.opensearch.migrations.MetadataArgs; import org.opensearch.migrations.MetadataMigration; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; -import static org.mockito.Mockito.mock; class ConfigureTest { @Test void configureSource_notImplemented() { - var meta = new MetadataMigration(mock(MetadataArgs.class)); + var meta = new MetadataMigration(); var configureSource = meta.configure() .execute(); diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/EvaluateTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/EvaluateTest.java index edbf6c1fc..9da7d6593 100644 --- a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/EvaluateTest.java +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/EvaluateTest.java @@ -2,8 +2,9 @@ import org.junit.jupiter.api.Test; -import org.opensearch.migrations.MetadataArgs; import org.opensearch.migrations.MetadataMigration; +import org.opensearch.migrations.Version; +import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; @@ -12,12 +13,16 @@ class EvaluateTest { @Test - void evaluate_notImplemented() { - var meta = new MetadataMigration(mock(MetadataArgs.class)); + void evaluate_failsUnexpectedException() { + var args = new EvaluateArgs(); + args.sourceVersion = Version.fromString("ES 7.10"); + args.fileSystemRepoPath = ""; - var configureSource = meta.evaluate() - .execute(); + var meta = new MetadataMigration(); + var context = mock(RootMetadataMigrationContext.class); + + var results = meta.evaluate(args).execute(context); - assertThat(configureSource.getExitCode(), equalTo(9999)); + assertThat(results.getExitCode(), equalTo(Evaluate.UNEXPECTED_FAILURE_CODE)); } } diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrateTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrateTest.java index 5b4abdb24..b3013388e 100644 --- a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrateTest.java +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrateTest.java @@ -2,7 +2,6 @@ import org.junit.jupiter.api.Test; -import org.opensearch.migrations.MetadataArgs; import org.opensearch.migrations.MetadataMigration; import org.opensearch.migrations.Version; import org.opensearch.migrations.metadata.tracing.RootMetadataMigrationContext; @@ -15,24 +14,24 @@ class MigrateTest { @Test void migrate_failsInvalidParameters() { - var args = new MetadataArgs(); + var args = new MigrateArgs(); var context = mock(RootMetadataMigrationContext.class); - var meta = new MetadataMigration(args); + var meta = new MetadataMigration(); - var configureSource = meta.migrate().execute(context); + var configureSource = meta.migrate(args).execute(context); assertThat(configureSource.getExitCode(), equalTo(Migrate.INVALID_PARAMETER_CODE)); } @Test void migrate_failsUnexpectedException() { - var args = new MetadataArgs(); + var args = new MigrateArgs(); args.sourceVersion = Version.fromString("ES 7.10"); args.fileSystemRepoPath = ""; var context = mock(RootMetadataMigrationContext.class); - var meta = new MetadataMigration(args); + var meta = new MetadataMigration(); - var configureSource = meta.migrate().execute(context); + var configureSource = meta.migrate(args).execute(context); assertThat(configureSource.getExitCode(), equalTo(Migrate.UNEXPECTED_FAILURE_CODE)); } diff --git a/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrationItemResultTest.java b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrationItemResultTest.java new file mode 100644 index 000000000..f5e745e42 --- /dev/null +++ b/MetadataMigration/src/test/java/org/opensearch/migrations/commands/MigrationItemResultTest.java @@ -0,0 +1,75 @@ +package org.opensearch.migrations.commands; + +import org.junit.jupiter.api.Test; + +import org.opensearch.migrations.cli.Clusters; +import org.opensearch.migrations.cli.Items; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; + +public class MigrationItemResultTest { + @Test + void testAsString_fullResults_withMessage() { + var clusters = mock(Clusters.class); + var items = mock(Items.class); + var testObject = EvaluateResult.builder() + .clusters(clusters) + .items(items) + .exitCode(10) + .errorMessage("Full results") + .build(); + + var result = testObject.asCliOutput(); + assertThat(result, containsString("Issue(s) detected")); + assertThat(result, containsString("Issues:")); + + verify(clusters).asCliOutput(); + verify(items).asCliOutput(); + verifyNoMoreInteractions(items, clusters); + } + + @Test + void testAsString_fullResults_withNoMessage() { + var clusters = mock(Clusters.class); + var items = mock(Items.class); + var testObject = EvaluateResult.builder() + .clusters(clusters) + .items(items) + .exitCode(10) + .build(); + + var result = testObject.asCliOutput(); + assertThat(result, containsString("10 issue(s) detected")); + verify(clusters).asCliOutput(); + verify(items).asCliOutput(); + verifyNoMoreInteractions(items, clusters); + } + + @Test + void testAsString_noItems() { + var clusters = mock(Clusters.class); + var testObject = EvaluateResult.builder() + .clusters(clusters) + .exitCode(0) + .build(); + + var result = testObject.asCliOutput(); + assertThat(result, containsString("0 issue(s) detected")); + verify(clusters).asCliOutput(); + verifyNoMoreInteractions(clusters); + } + + @Test + void testAsString_nothing() { + var testObject = EvaluateResult.builder() + .exitCode(0) + .build(); + + var result = testObject.asCliOutput(); + assertThat(result, containsString("0 issue(s) detected")); + } +} diff --git a/MetadataMigration/src/test/resources/log4j2.properties b/MetadataMigration/src/test/resources/log4j2.properties index 61c99871f..7171a6322 100644 --- a/MetadataMigration/src/test/resources/log4j2.properties +++ b/MetadataMigration/src/test/resources/log4j2.properties @@ -8,11 +8,16 @@ appender.console.target = SYSTEM_OUT appender.console.layout.type = PatternLayout appender.console.layout.pattern = %m%n +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-DEBUG} + rootLogger.level = info rootLogger.appenderRef.console.ref = Console +# Allow customization of owned package logs logger.rfs.name = com.rfs -logger.rfs.level = debug +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} logger.migrations.name = com.opensearch.migrations logger.migrations.level = debug @@ -32,4 +37,4 @@ logger.dockerclientdeps.level = info logger.wireLogger.name = org.apache.http.wire logger.wireLogger.level = OFF -logger.wireLogger.additivity = false \ No newline at end of file +logger.wireLogger.additivity = false diff --git a/RFS/build.gradle b/RFS/build.gradle index ca41bf739..6524f1ded 100644 --- a/RFS/build.gradle +++ b/RFS/build.gradle @@ -20,7 +20,7 @@ dependencies { implementation project(':coreUtilities') implementation project(':transformation') - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-annotations' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-core' @@ -31,7 +31,7 @@ dependencies { implementation group: 'org.apache.logging.log4j', name: 'log4j-core' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' implementation group: 'org.apache.lucene', name: 'lucene-core' - implementation group: 'org.apache.lucene', name: 'lucene-analyzers-common' + implementation group: 'org.apache.lucene', name: 'lucene-analysis-common' implementation group: 'org.apache.lucene', name: 'lucene-backward-codecs' implementation group: 'software.amazon.awssdk', name: 's3' implementation group: 'software.amazon.awssdk', name: 's3-transfer-manager' @@ -49,6 +49,7 @@ dependencies { testImplementation 'com.github.docker-java:docker-java-transport-httpclient5:3.3.6' testImplementation testFixtures(project(path: ':RFS')) + testImplementation group: 'org.apache.lucene', name: 'lucene-backward-codecs' testImplementation group: 'io.projectreactor', name: 'reactor-test' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-core' testImplementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' diff --git a/RFS/gradle.properties b/RFS/gradle.properties deleted file mode 100644 index 18f452c73..000000000 --- a/RFS/gradle.properties +++ /dev/null @@ -1,6 +0,0 @@ -# This file was generated by the Gradle 'init' task. -# https://docs.gradle.org/current/userguide/build_environment.html#sec:gradle_configuration_properties - -org.gradle.parallel=true -org.gradle.caching=true - diff --git a/RFS/src/main/java/com/rfs/common/DocumentReindexer.java b/RFS/src/main/java/com/rfs/common/DocumentReindexer.java index 85a5ef710..2110eceb5 100644 --- a/RFS/src/main/java/com/rfs/common/DocumentReindexer.java +++ b/RFS/src/main/java/com/rfs/common/DocumentReindexer.java @@ -5,8 +5,6 @@ import java.util.UUID; import java.util.function.Predicate; -import org.apache.lucene.document.Document; - import org.opensearch.migrations.reindexer.tracing.IDocumentMigrationContexts.IDocumentReindexContext; import lombok.EqualsAndHashCode; @@ -28,7 +26,7 @@ public class DocumentReindexer { private final long maxBytesPerBulkRequest; private final int maxConcurrentWorkItems; - public Mono reindex(String indexName, Flux documentStream, IDocumentReindexContext context) { + public Mono reindex(String indexName, Flux documentStream, IDocumentReindexContext context) { var scheduler = Schedulers.newParallel("DocumentBulkAggregator"); var bulkDocs = documentStream .publishOn(scheduler, 1) @@ -98,15 +96,15 @@ public static class BulkDocSection { private final String docId; private final String bulkIndex; - public BulkDocSection(Document doc) { - this.docId = Uid.decodeId(doc.getBinaryValue("_id").bytes); + public BulkDocSection(RfsLuceneDocument doc) { + this.docId = doc.id; this.bulkIndex = createBulkIndex(docId, doc); } @SneakyThrows - private static String createBulkIndex(final String docId, final Document doc) { + private static String createBulkIndex(final String docId, final RfsLuceneDocument doc) { // For a successful bulk ingestion, we cannot have any leading or trailing whitespace, and must be on a single line. - String trimmedSource = doc.getBinaryValue("_source").utf8ToString().trim().replace("\n", ""); + String trimmedSource = doc.source.trim().replace("\n", ""); return "{\"index\":{\"_id\":\"" + docId + "\"}}" + "\n" + trimmedSource; } diff --git a/RFS/src/main/java/com/rfs/common/LuceneDocumentsReader.java b/RFS/src/main/java/com/rfs/common/LuceneDocumentsReader.java index 265f5beb4..11b02fe6e 100644 --- a/RFS/src/main/java/com/rfs/common/LuceneDocumentsReader.java +++ b/RFS/src/main/java/com/rfs/common/LuceneDocumentsReader.java @@ -2,17 +2,21 @@ import java.io.IOException; import java.nio.file.Path; +import java.util.List; import java.util.concurrent.Callable; import java.util.function.Function; import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SoftDeletesDirectoryReaderWrapper; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.BytesRef; +import org.opensearch.migrations.cluster.ClusterSnapshotReader; + import lombok.Lombok; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -24,8 +28,13 @@ @RequiredArgsConstructor @Slf4j public class LuceneDocumentsReader { - public static Function getFactory(boolean softDeletesPossible, String softDeletesField) { - return path -> new LuceneDocumentsReader(path, softDeletesPossible, softDeletesField); + + public static Function getFactory(ClusterSnapshotReader snapshotReader) { + return path -> new LuceneDocumentsReader( + path, + snapshotReader.getSoftDeletesPossible(), + snapshotReader.getSoftDeletesFieldData() + ); } protected final Path indexDirectoryPath; @@ -79,7 +88,7 @@ public static Function getFactory(boolean softDelet * Lucene Index. */ - public Flux readDocuments() { + public Flux readDocuments() { return Flux.using( () -> wrapReader(getReader(), softDeletesPossible, softDeletesField), this::readDocsByLeavesInParallel, @@ -93,11 +102,20 @@ public Flux readDocuments() { }); } - protected DirectoryReader getReader() throws IOException { - return DirectoryReader.open(FSDirectory.open(indexDirectoryPath)); + protected DirectoryReader getReader() throws IOException {// Get the list of commits and pick the latest one + try (FSDirectory directory = FSDirectory.open(indexDirectoryPath)) { + List commits = DirectoryReader.listCommits(directory); + IndexCommit latestCommit = commits.get(commits.size() - 1); + + return DirectoryReader.open( + latestCommit, + 6, // Minimum supported major version - Elastic 5/Lucene 6 + null // No specific sorting required + ); + } } - Publisher readDocsByLeavesInParallel(DirectoryReader reader) { + Publisher readDocsByLeavesInParallel(DirectoryReader reader) { var segmentsToReadAtOnce = 5; // Arbitrary value var maxDocumentsToReadAtOnce = 100; // Arbitrary value log.atInfo().setMessage("{} documents in {} leaves found in the current Lucene index") @@ -116,7 +134,7 @@ Publisher readDocsByLeavesInParallel(DirectoryReader reader) { .doOnTerminate(sharedSegmentReaderScheduler::dispose); } - Publisher> getReadDocCallablesFromSegments(LeafReaderContext leafReaderContext) { + Publisher> getReadDocCallablesFromSegments(LeafReaderContext leafReaderContext) { @SuppressWarnings("resource") // segmentReader will be closed by parent DirectoryReader var segmentReader = leafReaderContext.reader(); var liveDocs = segmentReader.getLiveDocs(); @@ -135,18 +153,43 @@ protected DirectoryReader wrapReader(DirectoryReader reader, boolean softDeletes return reader; } - protected Document getDocument(IndexReader reader, int docId, boolean isLive) { + protected RfsLuceneDocument getDocument(IndexReader reader, int docId, boolean isLive) { try { Document document = reader.document(docId); - BytesRef sourceBytes = document.getBinaryValue("_source"); - String id; + String id = null; + BytesRef sourceBytes = null; try { - var idValue = document.getBinaryValue("_id"); - if (idValue == null) { + for (var field : document.getFields()) { + String fieldName = field.name(); + switch (fieldName) { + case "_id": { + // ES 6+ + var idBytes = field.binaryValue(); + id = Uid.decodeId(idBytes.bytes); + break; + } + case "_uid": { + // ES 5 + id = field.stringValue(); + break; + } + case "_source": { + // All versions (?) + sourceBytes = field.binaryValue(); + break; + } + } + } + if (id == null) { log.atError().setMessage("Document with index" + docId + " does not have an id. Skipping").log(); return null; // Skip documents with missing id } - id = Uid.decodeId(idValue.bytes); + + if (sourceBytes == null || sourceBytes.bytes.length == 0) { + log.atWarn().setMessage("Document {} doesn't have the _source field enabled").addArgument(id).log(); + return null; // Skip these + } + log.atDebug().setMessage("Reading document {}").addArgument(id).log(); } catch (Exception e) { StringBuilder errorMessage = new StringBuilder(); @@ -161,13 +204,8 @@ protected Document getDocument(IndexReader reader, int docId, boolean isLive) { return null; // Skip these } - if (sourceBytes == null || sourceBytes.bytes.length == 0) { - log.atWarn().setMessage("Document {} doesn't have the _source field enabled").addArgument(id).log(); - return null; // Skip these - } - log.atDebug().setMessage("Document {} read successfully").addArgument(id).log(); - return document; + return new RfsLuceneDocument(id, sourceBytes.utf8ToString()); } catch (Exception e) { log.atError().setMessage("Failed to read document at Lucene index location {}").addArgument(docId).setCause(e).log(); return null; diff --git a/RFS/src/main/java/com/rfs/common/OpenSearchClient.java b/RFS/src/main/java/com/rfs/common/OpenSearchClient.java index 0e9580ee4..f63d11da4 100644 --- a/RFS/src/main/java/com/rfs/common/OpenSearchClient.java +++ b/RFS/src/main/java/com/rfs/common/OpenSearchClient.java @@ -95,12 +95,12 @@ private Version versionFromResponse(HttpResponse resp) { .major(Integer.parseInt(parts[0])) .minor(Integer.parseInt(parts[1])) .patch(parts.length > 2 ? Integer.parseInt(parts[2]) : 0); - + var distroNode = versionNode.get("distribution"); if (distroNode != null && distroNode.asText().equalsIgnoreCase("opensearch")) { - versionBuilder.flavor(Flavor.OpenSearch); - } else { - versionBuilder.flavor(Flavor.Elasticsearch); + versionBuilder.flavor(Flavor.OPENSEARCH); + } else { + versionBuilder.flavor(Flavor.ELASTICSEARCH); } return versionBuilder.build(); } catch (Exception e) { @@ -148,6 +148,29 @@ public Optional createIndexTemplate( return createObjectIdempotent(targetPath, settings, context); } + /** Returns true if this template already exists */ + public boolean hasLegacyTemplate(String templateName) { + var targetPath = "_template/" + templateName; + return hasObjectCheck(targetPath, null); + } + + /** Returns true if this template already exists */ + public boolean hasComponentTemplate(String templateName) { + var targetPath = "_component_template/" + templateName; + return hasObjectCheck(targetPath, null); + } + + /** Returns true if this template already exists */ + public boolean hasIndexTemplate(String templateName) { + var targetPath = "_index_template/" + templateName; + return hasObjectCheck(targetPath, null); + } + + /** Returns true if this index already exists */ + public boolean hasIndex(String indexName) { + return hasObjectCheck(indexName, null); + } + /* * Create an index if it does not already exist. Returns an Optional; if the index was created, it * will be the created object and empty otherwise. @@ -166,29 +189,7 @@ private Optional createObjectIdempotent( ObjectNode settings, IRfsContexts.ICheckedIdempotentPutRequestContext context ) { - HttpResponse getResponse = client.getAsync(objectPath, context.createCheckRequestContext()) - .flatMap(resp -> { - if (resp.statusCode == HttpURLConnection.HTTP_NOT_FOUND || resp.statusCode == HttpURLConnection.HTTP_OK) { - return Mono.just(resp); - } else { - String errorMessage = ("Could not create object: " - + objectPath - + ". Response Code: " - + resp.statusCode - + ", Response Message: " - + resp.statusText - + ", Response Body: " - + resp.body); - return Mono.error(new OperationFailed(errorMessage, resp)); - } - }) - .doOnError(e -> log.error(e.getMessage())) - .retryWhen(checkIfItemExistsRetryStrategy) - .block(); - - assert getResponse != null : ("getResponse should not be null; it should either be a valid response or an exception" - + " should have been thrown."); - boolean objectDoesNotExist = getResponse.statusCode == HttpURLConnection.HTTP_NOT_FOUND; + var objectDoesNotExist = !hasObjectCheck(objectPath, context); if (objectDoesNotExist) { client.putAsync(objectPath, settings.toString(), context.createCheckRequestContext()).flatMap(resp -> { if (resp.statusCode == HttpURLConnection.HTTP_OK) { @@ -214,11 +215,45 @@ private Optional createObjectIdempotent( .block(); return Optional.of(settings); + } else { + log.debug("Object at path {} already exists, not attempting to create.", objectPath); } // The only response code that can end up here is HTTP_OK, which means the object already existed return Optional.empty(); } + private boolean hasObjectCheck( + String objectPath, + IRfsContexts.ICheckedIdempotentPutRequestContext context + ) { + var requestContext = Optional.ofNullable(context) + .map(c -> c.createCheckRequestContext()) + .orElse(null); + var getResponse = client.getAsync(objectPath, requestContext) + .flatMap(resp -> { + if (resp.statusCode == HttpURLConnection.HTTP_NOT_FOUND || resp.statusCode == HttpURLConnection.HTTP_OK) { + return Mono.just(resp); + } else { + String errorMessage = ("Could not create object: " + + objectPath + + ". Response Code: " + + resp.statusCode + + ", Response Message: " + + resp.statusText + + ", Response Body: " + + resp.body); + return Mono.error(new OperationFailed(errorMessage, resp)); + } + }) + .doOnError(e -> log.error(e.getMessage())) + .retryWhen(checkIfItemExistsRetryStrategy) + .block(); + + assert getResponse != null : ("getResponse should not be null; it should either be a valid response or an exception" + + " should have been thrown."); + return getResponse.statusCode == HttpURLConnection.HTTP_OK; + } + /* * Attempts to register a snapshot repository; no-op if the repo already exists. */ diff --git a/RFS/src/main/java/com/rfs/common/RfsLuceneDocument.java b/RFS/src/main/java/com/rfs/common/RfsLuceneDocument.java new file mode 100644 index 000000000..a09225d82 --- /dev/null +++ b/RFS/src/main/java/com/rfs/common/RfsLuceneDocument.java @@ -0,0 +1,9 @@ +package com.rfs.common; + +import lombok.RequiredArgsConstructor; + +@RequiredArgsConstructor +public class RfsLuceneDocument { + public final String id; + public final String source; +} diff --git a/RFS/src/main/java/com/rfs/common/TryHandlePhaseFailure.java b/RFS/src/main/java/com/rfs/common/TryHandlePhaseFailure.java deleted file mode 100644 index 120b22bce..000000000 --- a/RFS/src/main/java/com/rfs/common/TryHandlePhaseFailure.java +++ /dev/null @@ -1,20 +0,0 @@ -package com.rfs.common; - -import lombok.extern.slf4j.Slf4j; - -@Slf4j -public class TryHandlePhaseFailure { - @FunctionalInterface - public interface TryBlock { - void run() throws Exception; - } - - public static void executeWithTryCatch(TryBlock tryBlock) throws Exception { - try { - tryBlock.run(); - } catch (Exception e) { - log.atError().setMessage("Unexpected error running RfsWorker").setCause(e).log(); - throw e; - } - } -} diff --git a/RFS/src/main/java/com/rfs/common/http/ConnectionContext.java b/RFS/src/main/java/com/rfs/common/http/ConnectionContext.java index f28fb1cf2..05eafe29a 100644 --- a/RFS/src/main/java/com/rfs/common/http/ConnectionContext.java +++ b/RFS/src/main/java/com/rfs/common/http/ConnectionContext.java @@ -6,6 +6,7 @@ import com.beust.jcommander.Parameter; import com.beust.jcommander.ParametersDelegate; +import lombok.EqualsAndHashCode; import lombok.Getter; import lombok.ToString; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; @@ -14,6 +15,7 @@ * Stores the connection context for an Elasticsearch/OpenSearch cluster */ @Getter +@EqualsAndHashCode(exclude={"requestTransformer"}) @ToString(exclude={"requestTransformer"}) public class ConnectionContext { public enum Protocol { @@ -140,7 +142,7 @@ public static class TargetAdvancedArgs { public boolean compressionEnabled = false; } - @Getter + @Getter public static class SourceArgs implements IParams { @Parameter(names = { "--source-host" }, description = "The source host and port (e.g. http://localhost:9200)", required = false) diff --git a/RFS/src/main/java/com/rfs/transformers/Transformer_ES_6_8_to_OS_2_11.java b/RFS/src/main/java/com/rfs/transformers/Transformer_ES_6_8_to_OS_2_11.java index 9c0af53bf..6810592e5 100644 --- a/RFS/src/main/java/com/rfs/transformers/Transformer_ES_6_8_to_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/transformers/Transformer_ES_6_8_to_OS_2_11.java @@ -41,7 +41,7 @@ public GlobalMetadata transformGlobalMetadata(GlobalMetadata globalData) { var templateCopy = (ObjectNode) template.getValue().deepCopy(); var indexTemplate = (Index) () -> templateCopy; transformIndex(indexTemplate, IndexType.Template); - templates.set(template.getKey(), indexTemplate.rawJson()); + templates.set(template.getKey(), indexTemplate.getRawJson()); }); newRoot.set("templates", templates); } @@ -65,12 +65,12 @@ public GlobalMetadata transformGlobalMetadata(GlobalMetadata globalData) { public IndexMetadata transformIndexMetadata(IndexMetadata index) { var copy = index.deepCopy(); transformIndex(copy, IndexType.Concrete); - return new IndexMetadataData_OS_2_11(copy.rawJson(), copy.getId(), copy.getName()); + return new IndexMetadataData_OS_2_11(copy.getRawJson(), copy.getId(), copy.getName()); } private void transformIndex(Index index, IndexType type) { - logger.debug("Original Object: " + index.rawJson().toString()); - var newRoot = index.rawJson(); + logger.debug("Original Object: " + index.getRawJson().toString()); + var newRoot = index.getRawJson(); switch (type) { case Concrete: diff --git a/RFS/src/main/java/com/rfs/transformers/Transformer_ES_7_10_OS_2_11.java b/RFS/src/main/java/com/rfs/transformers/Transformer_ES_7_10_OS_2_11.java index 2809c363b..31e268c70 100644 --- a/RFS/src/main/java/com/rfs/transformers/Transformer_ES_7_10_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/transformers/Transformer_ES_7_10_OS_2_11.java @@ -86,9 +86,9 @@ public GlobalMetadata transformGlobalMetadata(GlobalMetadata metaData) { @Override public IndexMetadata transformIndexMetadata(IndexMetadata indexData) { - logger.debug("Original Object: " + indexData.rawJson().toString()); + logger.debug("Original Object: " + indexData.getRawJson().toString()); var copy = indexData.deepCopy(); - var newRoot = copy.rawJson(); + var newRoot = copy.getRawJson(); TransformFunctions.removeIntermediateMappingsLevels(newRoot); diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/ElasticsearchConstants_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/ElasticsearchConstants_ES_6_8.java index a74e69f22..98856c6c5 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/ElasticsearchConstants_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/ElasticsearchConstants_ES_6_8.java @@ -1,11 +1,15 @@ package com.rfs.version_es_6_8; +import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.fasterxml.jackson.dataformat.smile.SmileGenerator; + public class ElasticsearchConstants_ES_6_8 { + private ElasticsearchConstants_ES_6_8() {} + public static final int BUFFER_SIZE_IN_BYTES; public static final SmileFactory SMILE_FACTORY; public static final String SOFT_DELETES_FIELD; @@ -18,12 +22,13 @@ public class ElasticsearchConstants_ES_6_8 { // Taken from: // https://github.com/elastic/elasticsearch/blob/6.8/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java#L55 - SmileFactory smileFactory = new SmileFactory(); - smileFactory.configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false); - smileFactory.configure(SmileFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false); - smileFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); - smileFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, false); - SMILE_FACTORY = smileFactory; + SMILE_FACTORY = SmileFactory.builder() + .configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false) + .configure(JsonFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false) + .build(); + + SMILE_FACTORY.disable(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT); + SMILE_FACTORY.disable(JsonParser.Feature.STRICT_DUPLICATE_DETECTION); // Soft Deletes were added in 7.0 SOFT_DELETES_FIELD = ""; diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/IndexMetadataData_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/IndexMetadataData_ES_6_8.java index 5e3264283..45ee64f82 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/IndexMetadataData_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/IndexMetadataData_ES_6_8.java @@ -5,30 +5,29 @@ import com.rfs.models.IndexMetadata; import com.rfs.transformers.TransformFunctions; +import lombok.Getter; public class IndexMetadataData_ES_6_8 implements IndexMetadata { - private ObjectNode root; + @Getter + private final ObjectNode rawJson; private ObjectNode mappings; private ObjectNode settings; - private String indexId; - private String indexName; + @Getter + private final String id; + @Getter + private final String name; public IndexMetadataData_ES_6_8(ObjectNode root, String indexId, String indexName) { - this.root = root; + this.rawJson = root; this.mappings = null; this.settings = null; - this.indexId = indexId; - this.indexName = indexName; + this.id = indexId; + this.name = indexName; } @Override public ObjectNode getAliases() { - return (ObjectNode) root.get("aliases"); - } - - @Override - public String getId() { - return indexId; + return (ObjectNode) rawJson.get("aliases"); } @Override @@ -37,17 +36,12 @@ public JsonNode getMappings() { return mappings; } - ObjectNode mappingsNode = (ObjectNode) root.get("mappings"); + ObjectNode mappingsNode = (ObjectNode) rawJson.get("mappings"); mappings = mappingsNode; return mappings; } - @Override - public String getName() { - return indexName; - } - @Override public int getNumberOfShards() { return this.getSettings().get("index").get("number_of_shards").asInt(); @@ -59,20 +53,15 @@ public ObjectNode getSettings() { return settings; } - ObjectNode treeSettings = TransformFunctions.convertFlatSettingsToTree((ObjectNode) root.get("settings")); + ObjectNode treeSettings = TransformFunctions.convertFlatSettingsToTree((ObjectNode) rawJson.get("settings")); settings = treeSettings; return settings; } - @Override - public ObjectNode rawJson() { - return root; - } - @Override public IndexMetadata deepCopy() { - return new IndexMetadataData_ES_6_8(root.deepCopy(), indexId, indexName); + return new IndexMetadataData_ES_6_8(rawJson.deepCopy(), id, name); } } diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/ShardMetadataData_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/ShardMetadataData_ES_6_8.java index caa97408a..0a5fbbd9a 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/ShardMetadataData_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/ShardMetadataData_ES_6_8.java @@ -1,13 +1,12 @@ package com.rfs.version_es_6_8; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.JsonNode; @@ -16,20 +15,22 @@ import com.rfs.models.ShardFileInfo; import com.rfs.models.ShardMetadata; +import lombok.Getter; +@Getter public class ShardMetadataData_ES_6_8 implements ShardMetadata { private static final ObjectMapper objectMapper = new ObjectMapper(); - private String snapshotName; - private String indexName; - private String indexId; - private int shardId; - private int indexVersion; - private long startTime; - private long time; - private int numberOfFiles; - private long totalSize; - private List files; + private final String snapshotName; + private final String indexName; + private final String indexId; + private final int shardId; + private final int indexVersion; + private final long startTime; + private final long time; + private final int numberOfFiles; + private final long totalSizeBytes; + private final List files; public ShardMetadataData_ES_6_8( String snapshotName, @@ -51,65 +52,14 @@ public ShardMetadataData_ES_6_8( this.startTime = startTime; this.time = time; this.numberOfFiles = numberOfFiles; - this.totalSize = totalSize; + this.totalSizeBytes = totalSize; // Convert the raw file metadata to the FileMetadata class List convertedFiles = new java.util.ArrayList<>(); for (FileInfoRaw fileMetadataRaw : files) { convertedFiles.add(FileInfo.fromFileMetadataRaw(fileMetadataRaw)); } - this.files = convertedFiles; - } - - @Override - public String getSnapshotName() { - return snapshotName; - } - - @Override - public String getIndexName() { - return indexName; - } - - @Override - public String getIndexId() { - return indexId; - } - - @Override - public int getShardId() { - return shardId; - } - - @Override - public int getIndexVersion() { - return indexVersion; - } - - @Override - public long getStartTime() { - return startTime; - } - - @Override - public long getTime() { - return time; - } - - @Override - public int getNumberOfFiles() { - return numberOfFiles; - } - - @Override - public long getTotalSizeBytes() { - return totalSize; - } - - @Override - public List getFiles() { - List convertedFiles = new ArrayList<>(files); - return convertedFiles; + this.files = Collections.unmodifiableList(convertedFiles); } @Override @@ -150,15 +100,16 @@ public DataRaw( } } + @Getter public static class FileInfo implements ShardFileInfo { - private String name; - private String physicalName; - private long length; - private String checksum; - private long partSize; - private long numberOfParts; - private String writtenBy; - private BytesRef metaHash; + private final String name; + private final String physicalName; + private final long length; + private final String checksum; + private final long partSize; + private final long numberOfParts; + private final String writtenBy; + private final BytesRef metaHash; public static FileInfo fromFileMetadataRaw(FileInfoRaw fileMetadataRaw) { return new FileInfo( @@ -189,7 +140,9 @@ public FileInfo( this.writtenBy = writtenBy; this.metaHash = metaHash; - // Calculate the number of parts the file is chopped into; taken from Elasticsearch code + // Calculate the number of parts the file is chopped into; taken from Elasticsearch code. When Elasticsearch makes + // a snapshot and finds Lucene files over a specified size, it will split those files into multiple parts based on the + // maximum part size. // See: // https://github.com/elastic/elasticsearch/blob/6.8/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java#L68 long partBytes = Long.MAX_VALUE; @@ -198,54 +151,14 @@ public FileInfo( } long totalLength = length; - long numberOfParts = totalLength / partBytes; + long numberOfPartsTemp = totalLength / partBytes; if (totalLength % partBytes > 0) { - numberOfParts++; + numberOfPartsTemp++; } - if (numberOfParts == 0) { - numberOfParts++; + if (numberOfPartsTemp == 0) { + numberOfPartsTemp++; } - this.numberOfParts = numberOfParts; - } - - @Override - public String getName() { - return name; - } - - @Override - public String getPhysicalName() { - return physicalName; - } - - @Override - public long getLength() { - return length; - } - - @Override - public String getChecksum() { - return checksum; - } - - @Override - public long getPartSize() { - return partSize; - } - - @Override - public String getWrittenBy() { - return writtenBy; - } - - @Override - public BytesRef getMetaHash() { - return metaHash; - } - - @Override - public long getNumberOfParts() { - return numberOfParts; + this.numberOfParts = numberOfPartsTemp; } // The Snapshot file may be split into multiple blobs; use this to find the correct file name @@ -298,8 +211,7 @@ public FileInfoRaw( public static class FileInfoRawDeserializer extends JsonDeserializer { @Override - public FileInfoRaw deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException, - JsonProcessingException { + public FileInfoRaw deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException { JsonNode rootNode = jp.getCodec().readTree(jp); diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataData_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataData_ES_6_8.java index ed78bb576..14f4b7b5c 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataData_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataData_ES_6_8.java @@ -5,9 +5,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.rfs.models.SnapshotMetadata; +import lombok.Getter; +@Getter public class SnapshotMetadataData_ES_6_8 implements SnapshotMetadata { - private String name; private String uuid; @JsonProperty("version_id") @@ -26,64 +27,4 @@ public class SnapshotMetadataData_ES_6_8 implements SnapshotMetadata { @JsonProperty("successful_shards") private int successfulShards; private List failures; // Haven't looked at this yet - - @Override - public String getName() { - return name; - } - - @Override - public String getUuid() { - return uuid; - } - - @Override - public int getVersionId() { - return versionId; - } - - @Override - public List getIndices() { - return indices; - } - - @Override - public String getState() { - return state; - } - - @Override - public String getReason() { - return reason; - } - - @Override - public boolean isIncludeGlobalState() { - return includeGlobalState; - } - - @Override - public long getStartTime() { - return startTime; - } - - @Override - public long getEndTime() { - return endTime; - } - - @Override - public int getTotalShards() { - return totalShards; - } - - @Override - public int getSuccessfulShards() { - return successfulShards; - } - - @Override - public List getFailures() { - return failures; - } } diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataFactory_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataFactory_ES_6_8.java index 34c8106d1..dc328faf7 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataFactory_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotMetadataFactory_ES_6_8.java @@ -18,11 +18,10 @@ public class SnapshotMetadataFactory_ES_6_8 implements SnapshotMetadata.Factory public SnapshotMetadata fromJsonNode(JsonNode root) throws Exception { ObjectMapper mapper = new ObjectMapper(); ObjectNode objectNodeRoot = (ObjectNode) root; - SnapshotMetadataData_ES_6_8 snapshotMetadata = mapper.treeToValue( + return mapper.treeToValue( objectNodeRoot.get("snapshot"), SnapshotMetadataData_ES_6_8.class ); - return snapshotMetadata; } @Override diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoData_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoData_ES_6_8.java index 74c453c80..8a34e929b 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoData_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoData_ES_6_8.java @@ -11,6 +11,10 @@ import com.rfs.common.SnapshotRepo; import com.rfs.common.SnapshotRepo.CantParseRepoFile; import com.rfs.common.SourceRepo; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.RequiredArgsConstructor; public class SnapshotRepoData_ES_6_8 { @@ -36,19 +40,20 @@ public static SnapshotRepoData_ES_6_8 fromRepo(SourceRepo repo) { return fromRepoFile(file); } - public Path filePath; - public List snapshots; - public Map indices; + @Getter + private Path filePath; + @Getter + private List snapshots; + @Getter + private Map indices; + @Getter + @AllArgsConstructor + @NoArgsConstructor public static class Snapshot implements SnapshotRepo.Snapshot { - public String name; - public String uuid; - public int state; - - @Override - public String getName() { - return name; - } + private String name; + private String uuid; + private int state; @Override public String getId() { @@ -56,37 +61,23 @@ public String getId() { } } + @Getter + @AllArgsConstructor + @NoArgsConstructor public static class RawIndex { - public String id; - public List snapshots; + private String id; + private List snapshots; } + @Getter + @RequiredArgsConstructor public static class Index implements SnapshotRepo.Index { public static Index fromRawIndex(String name, RawIndex rawIndex) { - Index index = new Index(); - index.name = name; - index.id = rawIndex.id; - index.snapshots = rawIndex.snapshots; - return index; - } - - public String name; - public String id; - public List snapshots; - - @Override - public String getName() { - return name; + return new Index(name, rawIndex.id, rawIndex.snapshots); } - @Override - public String getId() { - return id; - } - - @Override - public List getSnapshots() { - return snapshots; - } + private final String name; + private final String id; + private final List snapshots; } } diff --git a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoProvider_ES_6_8.java b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoProvider_ES_6_8.java index 7576b9757..44cef6202 100644 --- a/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoProvider_ES_6_8.java +++ b/RFS/src/main/java/com/rfs/version_es_6_8/SnapshotRepoProvider_ES_6_8.java @@ -23,7 +23,7 @@ protected SnapshotRepoData_ES_6_8 getRepoData() { } public List getIndices() { - return getRepoData().indices.entrySet() + return getRepoData().getIndices().entrySet() .stream() .map(entry -> SnapshotRepoData_ES_6_8.Index.fromRawIndex(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); @@ -32,14 +32,14 @@ public List getIndices() { @Override public List getIndicesInSnapshot(String snapshotName) { List matchedIndices = new ArrayList<>(); - SnapshotRepoData_ES_6_8.Snapshot targetSnapshot = getRepoData().snapshots.stream() - .filter(snapshot -> snapshotName.equals(snapshot.name)) + SnapshotRepoData_ES_6_8.Snapshot targetSnapshot = getRepoData().getSnapshots().stream() + .filter(snapshot -> snapshotName.equals(snapshot.getName())) .findFirst() .orElse(null); if (targetSnapshot != null) { - getRepoData().indices.forEach((indexName, rawIndex) -> { - if (rawIndex.snapshots.contains(targetSnapshot.uuid)) { + getRepoData().getIndices().forEach((indexName, rawIndex) -> { + if (rawIndex.getSnapshots().contains(targetSnapshot.getId())) { matchedIndices.add(SnapshotRepoData_ES_6_8.Index.fromRawIndex(indexName, rawIndex)); } }); @@ -49,15 +49,14 @@ public List getIndicesInSnapshot(String snapshotName) { @Override public List getSnapshots() { - List convertedList = new ArrayList<>(getRepoData().snapshots); - return convertedList; + return new ArrayList<>(getRepoData().getSnapshots()); } @Override public String getSnapshotId(String snapshotName) { - for (SnapshotRepoData_ES_6_8.Snapshot snapshot : getRepoData().snapshots) { - if (snapshot.name.equals(snapshotName)) { - return snapshot.uuid; + for (SnapshotRepoData_ES_6_8.Snapshot snapshot : getRepoData().getSnapshots()) { + if (snapshot.getName().equals(snapshotName)) { + return snapshot.getId(); } } return null; @@ -65,7 +64,7 @@ public String getSnapshotId(String snapshotName) { @Override public String getIndexId(String indexName) { - return getRepoData().indices.get(indexName).id; + return getRepoData().getIndices().get(indexName).getId(); } @Override diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/ElasticsearchConstants_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/ElasticsearchConstants_ES_7_10.java index e26b90244..3d95fb245 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/ElasticsearchConstants_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/ElasticsearchConstants_ES_7_10.java @@ -1,11 +1,15 @@ package com.rfs.version_es_7_10; +import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.fasterxml.jackson.dataformat.smile.SmileGenerator; + public class ElasticsearchConstants_ES_7_10 { + private ElasticsearchConstants_ES_7_10() {} + public static final int BUFFER_SIZE_IN_BYTES; public static final SmileFactory SMILE_FACTORY; public static final String SOFT_DELETES_FIELD; @@ -17,12 +21,13 @@ public class ElasticsearchConstants_ES_7_10 { // Taken from: // https://github.com/elastic/elasticsearch/blob/7.10/libs/x-content/src/main/java/org/elasticsearch/common/xcontent/smile/SmileXContent.java#L54 - SmileFactory smileFactory = new SmileFactory(); - smileFactory.configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false); - smileFactory.configure(SmileFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false); - smileFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); - smileFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, false); - SMILE_FACTORY = smileFactory; + SMILE_FACTORY = SmileFactory.builder() + .configure(SmileGenerator.Feature.ENCODE_BINARY_AS_7BIT, false) + .configure(JsonFactory.Feature.FAIL_ON_SYMBOL_HASH_OVERFLOW, false) + .build(); + + SMILE_FACTORY.disable(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT); + SMILE_FACTORY.disable(JsonParser.Feature.STRICT_DUPLICATE_DETECTION); // Taken from: // https://github.com/elastic/elasticsearch/blob/v7.10.2/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java#L110 diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/IndexMetadataData_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/IndexMetadataData_ES_7_10.java index 54ed58ffb..64d712628 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/IndexMetadataData_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/IndexMetadataData_ES_7_10.java @@ -4,30 +4,29 @@ import com.rfs.models.IndexMetadata; import com.rfs.transformers.TransformFunctions; +import lombok.Getter; public class IndexMetadataData_ES_7_10 implements IndexMetadata { - private ObjectNode root; + @Getter + private final ObjectNode rawJson; private ObjectNode mappings; private ObjectNode settings; - private String indexId; - private String indexName; + @Getter + private final String id; + @Getter + private final String name; public IndexMetadataData_ES_7_10(ObjectNode root, String indexId, String indexName) { - this.root = root; + this.rawJson = root; this.mappings = null; this.settings = null; - this.indexId = indexId; - this.indexName = indexName; + this.id = indexId; + this.name = indexName; } @Override public ObjectNode getAliases() { - return (ObjectNode) root.get("aliases"); - } - - @Override - public String getId() { - return indexId; + return (ObjectNode) rawJson.get("aliases"); } @Override @@ -36,17 +35,12 @@ public ObjectNode getMappings() { return mappings; } - ObjectNode mappingsNode = (ObjectNode) root.get("mappings"); + ObjectNode mappingsNode = (ObjectNode) rawJson.get("mappings"); mappings = mappingsNode; return mappings; } - @Override - public String getName() { - return indexName; - } - @Override public int getNumberOfShards() { return this.getSettings().get("index").get("number_of_shards").asInt(); @@ -58,20 +52,15 @@ public ObjectNode getSettings() { return settings; } - ObjectNode treeSettings = TransformFunctions.convertFlatSettingsToTree((ObjectNode) root.get("settings")); + ObjectNode treeSettings = TransformFunctions.convertFlatSettingsToTree((ObjectNode) rawJson.get("settings")); settings = treeSettings; return settings; } - @Override - public ObjectNode rawJson() { - return root; - } - @Override public IndexMetadata deepCopy() { - return new IndexMetadataData_ES_7_10(root.deepCopy(), indexId, indexName); + return new IndexMetadataData_ES_7_10(rawJson.deepCopy(), id, name); } } diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/ShardMetadataData_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/ShardMetadataData_ES_7_10.java index 102a91c4c..31a238b28 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/ShardMetadataData_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/ShardMetadataData_ES_7_10.java @@ -1,13 +1,12 @@ package com.rfs.version_es_7_10; import java.io.IOException; -import java.util.ArrayList; +import java.util.Collections; import java.util.List; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.JsonDeserializer; @@ -17,7 +16,9 @@ import com.rfs.models.ShardFileInfo; import com.rfs.models.ShardMetadata; +import lombok.Getter; +@Getter public class ShardMetadataData_ES_7_10 implements ShardMetadata { private static final ObjectMapper objectMapper = new ObjectMapper(); @@ -25,16 +26,16 @@ public class ShardMetadataData_ES_7_10 implements ShardMetadata { objectMapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); } - private String snapshotName; - private String indexName; - private String indexId; - private int shardId; - private int indexVersion; - private long startTime; - private long time; - private int numberOfFiles; - private long totalSize; - private List files; + private final String snapshotName; + private final String indexName; + private final String indexId; + private final int shardId; + private final int indexVersion; + private final long startTime; + private final long time; + private final int numberOfFiles; + private final long totalSizeBytes; + private final List files; public ShardMetadataData_ES_7_10( String snapshotName, @@ -56,65 +57,14 @@ public ShardMetadataData_ES_7_10( this.startTime = startTime; this.time = time; this.numberOfFiles = numberOfFiles; - this.totalSize = totalSize; + this.totalSizeBytes = totalSize; // Convert the raw file metadata to the FileMetadata class List convertedFiles = new java.util.ArrayList<>(); for (FileInfoRaw fileMetadataRaw : files) { convertedFiles.add(FileInfo.fromFileMetadataRaw(fileMetadataRaw)); } - this.files = convertedFiles; - } - - @Override - public String getSnapshotName() { - return snapshotName; - } - - @Override - public String getIndexName() { - return indexName; - } - - @Override - public String getIndexId() { - return indexId; - } - - @Override - public int getShardId() { - return shardId; - } - - @Override - public int getIndexVersion() { - return indexVersion; - } - - @Override - public long getStartTime() { - return startTime; - } - - @Override - public long getTime() { - return time; - } - - @Override - public int getNumberOfFiles() { - return numberOfFiles; - } - - @Override - public long getTotalSizeBytes() { - return totalSize; - } - - @Override - public List getFiles() { - List convertedFiles = new ArrayList<>(files); - return convertedFiles; + this.files = Collections.unmodifiableList(convertedFiles); } @Override @@ -155,15 +105,16 @@ public DataRaw( } } + @Getter public static class FileInfo implements ShardFileInfo { - private String name; - private String physicalName; - private long length; - private String checksum; - private long partSize; - private long numberOfParts; - private String writtenBy; - private BytesRef metaHash; + private final String name; + private final String physicalName; + private final long length; + private final String checksum; + private final long partSize; + private final long numberOfParts; + private final String writtenBy; + private final BytesRef metaHash; public static FileInfo fromFileMetadataRaw(FileInfoRaw fileMetadataRaw) { return new FileInfo( @@ -194,7 +145,9 @@ public FileInfo( this.writtenBy = writtenBy; this.metaHash = metaHash; - // Calculate the number of parts the file is chopped into; taken from Elasticsearch code + // Calculate the number of parts the file is chopped into; taken from Elasticsearch code. When Elasticsearch makes + // a snapshot and finds Lucene files over a specified size, it will split those files into multiple parts based on the + // maximum part size. // See: // https://github.com/elastic/elasticsearch/blob/6.8/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java#L68 long partBytes = Long.MAX_VALUE; @@ -203,54 +156,14 @@ public FileInfo( } long totalLength = length; - long numberOfParts = totalLength / partBytes; + long numberOfPartsTemp = totalLength / partBytes; if (totalLength % partBytes > 0) { - numberOfParts++; + numberOfPartsTemp++; } - if (numberOfParts == 0) { - numberOfParts++; + if (numberOfPartsTemp == 0) { + numberOfPartsTemp++; } - this.numberOfParts = numberOfParts; - } - - @Override - public String getName() { - return name; - } - - @Override - public String getPhysicalName() { - return physicalName; - } - - @Override - public long getLength() { - return length; - } - - @Override - public String getChecksum() { - return checksum; - } - - @Override - public long getPartSize() { - return partSize; - } - - @Override - public String getWrittenBy() { - return writtenBy; - } - - @Override - public BytesRef getMetaHash() { - return metaHash; - } - - @Override - public long getNumberOfParts() { - return numberOfParts; + this.numberOfParts = numberOfPartsTemp; } // The Snapshot file may be split into multiple blobs; use this to find the correct file name @@ -303,8 +216,7 @@ public FileInfoRaw( public static class FileInfoRawDeserializer extends JsonDeserializer { @Override - public FileInfoRaw deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException, - JsonProcessingException { + public FileInfoRaw deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException { JsonNode rootNode = jp.getCodec().readTree(jp); diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataData_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataData_ES_7_10.java index 9a609ef9d..09912837d 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataData_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataData_ES_7_10.java @@ -5,9 +5,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.rfs.models.SnapshotMetadata; +import lombok.Getter; +@Getter public class SnapshotMetadataData_ES_7_10 implements SnapshotMetadata { - private String name; private String uuid; @JsonProperty("version_id") @@ -30,64 +31,4 @@ public class SnapshotMetadataData_ES_7_10 implements SnapshotMetadata { private List dataStreams; // Haven't looked into this yet @JsonProperty("metadata") private Object metaData; // Haven't looked into this yet - - @Override - public String getName() { - return name; - } - - @Override - public String getUuid() { - return uuid; - } - - @Override - public int getVersionId() { - return versionId; - } - - @Override - public List getIndices() { - return indices; - } - - @Override - public String getState() { - return state; - } - - @Override - public String getReason() { - return reason; - } - - @Override - public boolean isIncludeGlobalState() { - return includeGlobalState; - } - - @Override - public long getStartTime() { - return startTime; - } - - @Override - public long getEndTime() { - return endTime; - } - - @Override - public int getTotalShards() { - return totalShards; - } - - @Override - public int getSuccessfulShards() { - return successfulShards; - } - - @Override - public List getFailures() { - return failures; - } } diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataFactory_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataFactory_ES_7_10.java index e79b774e8..565803e0f 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataFactory_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotMetadataFactory_ES_7_10.java @@ -20,11 +20,10 @@ public SnapshotMetadata fromJsonNode(JsonNode root) throws Exception { ObjectMapper mapper = new ObjectMapper(); mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); ObjectNode objectNodeRoot = (ObjectNode) root; - SnapshotMetadataData_ES_7_10 snapshotMetadata = mapper.treeToValue( + return mapper.treeToValue( objectNodeRoot.get("snapshot"), SnapshotMetadataData_ES_7_10.class ); - return snapshotMetadata; } @Override diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10.java index b7b95f77d..d98a62f1e 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10.java @@ -13,7 +13,12 @@ import com.rfs.common.SnapshotRepo; import com.rfs.common.SnapshotRepo.CantParseRepoFile; import com.rfs.common.SourceRepo; +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.NoArgsConstructor; +import lombok.RequiredArgsConstructor; +@Getter public class SnapshotRepoData_ES_7_10 { public static SnapshotRepoData_ES_7_10 fromRepoFile(Path filePath) { ObjectMapper mapper = new ObjectMapper(); @@ -38,26 +43,24 @@ public static SnapshotRepoData_ES_7_10 fromRepo(SourceRepo repo) { return fromRepoFile(file); } - public Path filePath; - public List snapshots; - public Map indices; + private Path filePath; + private List snapshots; + private Map indices; @JsonProperty("min_version") - public String minVersion; + private String minVersion; @JsonProperty("index_metadata_identifiers") - public Map indexMetadataIdentifiers; + private Map indexMetadataIdentifiers; + @Getter + @AllArgsConstructor + @NoArgsConstructor public static class Snapshot implements SnapshotRepo.Snapshot { - public String name; - public String uuid; - public int state; + private String name; + private String uuid; + private int state; @JsonProperty("index_metadata_lookup") - public Map indexMetadataLookup; - public String version; - - @Override - public String getName() { - return name; - } + private Map indexMetadataLookup; + private String version; @Override public String getId() { @@ -65,41 +68,26 @@ public String getId() { } } + @Getter + @AllArgsConstructor + @NoArgsConstructor public static class RawIndex { - public String id; - public List snapshots; + private String id; + private List snapshots; @JsonProperty("shard_generations") - public List shardGenerations; + private List shardGenerations; } + @Getter + @RequiredArgsConstructor public static class Index implements SnapshotRepo.Index { public static Index fromRawIndex(String name, RawIndex rawIndex) { - Index index = new Index(); - index.name = name; - index.id = rawIndex.id; - index.snapshots = rawIndex.snapshots; - index.shardGenerations = rawIndex.shardGenerations; - return index; - } - - public String name; - public String id; - public List snapshots; - public List shardGenerations; - - @Override - public String getName() { - return name; + return new Index(name, rawIndex.id, rawIndex.snapshots, rawIndex.shardGenerations); } - @Override - public String getId() { - return id; - } - - @Override - public List getSnapshots() { - return snapshots; - } + private final String name; + private final String id; + private final List snapshots; + private final List shardGenerations; } } diff --git a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoProvider_ES_7_10.java b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoProvider_ES_7_10.java index 046c1a916..ab154f85e 100644 --- a/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoProvider_ES_7_10.java +++ b/RFS/src/main/java/com/rfs/version_es_7_10/SnapshotRepoProvider_ES_7_10.java @@ -23,7 +23,7 @@ protected SnapshotRepoData_ES_7_10 getRepoData() { } public List getIndices() { - return getRepoData().indices.entrySet() + return getRepoData().getIndices().entrySet() .stream() .map(entry -> SnapshotRepoData_ES_7_10.Index.fromRawIndex(entry.getKey(), entry.getValue())) .collect(Collectors.toList()); @@ -32,15 +32,15 @@ public List getIndices() { @Override public List getIndicesInSnapshot(String snapshotName) { List matchedIndices = new ArrayList<>(); - SnapshotRepoData_ES_7_10.Snapshot targetSnapshot = getRepoData().snapshots.stream() - .filter(snapshot -> snapshotName.equals(snapshot.name)) + SnapshotRepoData_ES_7_10.Snapshot targetSnapshot = getRepoData().getSnapshots().stream() + .filter(snapshot -> snapshotName.equals(snapshot.getName())) .findFirst() .orElse(null); if (targetSnapshot != null) { - targetSnapshot.indexMetadataLookup.keySet().forEach(indexId -> { - getRepoData().indices.forEach((indexName, rawIndex) -> { - if (indexId.equals(rawIndex.id)) { + targetSnapshot.getIndexMetadataLookup().keySet().forEach(indexId -> { + getRepoData().getIndices().forEach((indexName, rawIndex) -> { + if (indexId.equals(rawIndex.getId())) { matchedIndices.add(SnapshotRepoData_ES_7_10.Index.fromRawIndex(indexName, rawIndex)); } }); @@ -51,14 +51,13 @@ public List getIndicesInSnapshot(String snapshotName) { @Override public List getSnapshots() { - List convertedList = new ArrayList<>(getRepoData().snapshots); - return convertedList; + return new ArrayList<>(getRepoData().getSnapshots()); } public String getSnapshotId(String snapshotName) { - for (SnapshotRepoData_ES_7_10.Snapshot snapshot : getRepoData().snapshots) { - if (snapshot.name.equals(snapshotName)) { - return snapshot.uuid; + for (SnapshotRepoData_ES_7_10.Snapshot snapshot : getRepoData().getSnapshots()) { + if (snapshot.getName().equals(snapshotName)) { + return snapshot.getId(); } } return null; @@ -66,7 +65,7 @@ public String getSnapshotId(String snapshotName) { @Override public String getIndexId(String indexName) { - return getRepoData().indices.get(indexName).id; + return getRepoData().getIndices().get(indexName).getId(); } @Override @@ -80,15 +79,15 @@ public String getIndexMetadataId(String snapshotName, String indexName) { return null; } - String metadataLookupKey = getRepoData().snapshots.stream() - .filter(snapshot -> snapshot.name.equals(snapshotName)) - .map(snapshot -> snapshot.indexMetadataLookup.get(indexId)) + String metadataLookupKey = getRepoData().getSnapshots().stream() + .filter(snapshot -> snapshot.getName().equals(snapshotName)) + .map(snapshot -> snapshot.getIndexMetadataLookup().get(indexId)) .findFirst() .orElse(null); if (metadataLookupKey == null) { return null; } - return getRepoData().indexMetadataIdentifiers.get(metadataLookupKey); + return getRepoData().getIndexMetadataIdentifiers().get(metadataLookupKey); } } diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java index ace22e1bd..4ffb4e0f6 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/GlobalMetadataCreator_OS_2_11.java @@ -1,12 +1,13 @@ package com.rfs.version_os_2_11; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Optional; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.GlobalMetadataCreator; import org.opensearch.migrations.metadata.GlobalMetadataCreatorResults; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.IClusterMetadataContext; @@ -14,101 +15,120 @@ import com.rfs.common.OpenSearchClient; import com.rfs.models.GlobalMetadata; import lombok.AllArgsConstructor; +import lombok.extern.slf4j.Slf4j; @AllArgsConstructor +@Slf4j public class GlobalMetadataCreator_OS_2_11 implements GlobalMetadataCreator { - private static final Logger logger = LogManager.getLogger(GlobalMetadataCreator_OS_2_11.class); private final OpenSearchClient client; private final List legacyTemplateAllowlist; private final List componentTemplateAllowlist; private final List indexTemplateAllowlist; - public GlobalMetadataCreatorResults create(GlobalMetadata root, IClusterMetadataContext context) { - logger.info("Setting Global Metadata"); + public GlobalMetadataCreatorResults create( + GlobalMetadata root, + MigrationMode mode, + IClusterMetadataContext context) { + log.info("Setting Global Metadata"); var results = GlobalMetadataCreatorResults.builder(); GlobalMetadataData_OS_2_11 globalMetadata = new GlobalMetadataData_OS_2_11(root.toObjectNode()); - results.legacyTemplates(createLegacyTemplates(globalMetadata, client, legacyTemplateAllowlist, context)); - results.componentTemplates(createComponentTemplates(globalMetadata, client, componentTemplateAllowlist, context)); - results.indexTemplates(createIndexTemplates(globalMetadata, client, indexTemplateAllowlist, context)); + results.legacyTemplates(createLegacyTemplates(globalMetadata, mode, context)); + results.componentTemplates(createComponentTemplates(globalMetadata, mode, context)); + results.indexTemplates(createIndexTemplates(globalMetadata, mode, context)); return results.build(); } - protected List createLegacyTemplates( - GlobalMetadataData_OS_2_11 globalMetadata, - OpenSearchClient client, - List templateAllowlist, - IClusterMetadataContext context - ) { - var legacyTemplates = new ArrayList(); - logger.info("Setting Legacy Templates..."); - ObjectNode templates = globalMetadata.getTemplates(); + public List createLegacyTemplates(GlobalMetadataData_OS_2_11 metadata, MigrationMode mode, IClusterMetadataContext context) { + return createTemplates( + metadata.getTemplates(), + legacyTemplateAllowlist, + TemplateTypes.LegacyIndexTemplate, + mode, + context + ); + } - if (templates == null) { - logger.info("No Legacy Templates in Snapshot"); - return legacyTemplates; - } + public List createComponentTemplates(GlobalMetadataData_OS_2_11 metadata, MigrationMode mode, IClusterMetadataContext context) { + return createTemplates( + metadata.getComponentTemplates(), + componentTemplateAllowlist, + TemplateTypes.ComponentTemplates, + mode, + context + ); + } - if (templateAllowlist != null && templateAllowlist.size() == 0) { - logger.info("No Legacy Templates in specified allowlist"); - } else if (templateAllowlist != null) { - for (String templateName : templateAllowlist) { - if (!templates.has(templateName) || templates.get(templateName) == null) { - logger.warn("Legacy Template not found: " + templateName); - continue; - } + public List createIndexTemplates(GlobalMetadataData_OS_2_11 metadata, MigrationMode mode, IClusterMetadataContext context) { + return createTemplates( + metadata.getIndexTemplates(), + indexTemplateAllowlist, + TemplateTypes.IndexTemplate, + mode, + context + ); + } - logger.info("Setting Legacy Template: " + templateName); - ObjectNode settings = (ObjectNode) globalMetadata.getTemplates().get(templateName); - client.createLegacyTemplate(templateName, settings, context.createMigrateLegacyTemplateContext()); - legacyTemplates.add(templateName); - } - } else { - // Get the template names - List templateKeys = new ArrayList<>(); - templates.fieldNames().forEachRemaining(templateKeys::add); + @AllArgsConstructor + private enum TemplateTypes { + IndexTemplate( + (client, name, body, context) -> client.createIndexTemplate(name, body, context.createMigrateTemplateContext()), + (client, name) -> client.hasIndexTemplate(name) + ), + + LegacyIndexTemplate( + (client, name, body, context) -> client.createLegacyTemplate(name, body, context.createMigrateLegacyTemplateContext()), + (client, name) -> client.hasLegacyTemplate(name) + ), + + ComponentTemplates( + (client, name, body, context) -> client.createComponentTemplate(name, body, context.createComponentTemplateContext()), + (client, name) -> client.hasComponentTemplate(name) + ); + final TemplateCreator creator; + final TemplateExistsCheck alreadyExistsCheck; + } - // Create each template - for (String templateName : templateKeys) { - logger.info("Setting Legacy Template: " + templateName); - ObjectNode settings = (ObjectNode) templates.get(templateName); - client.createLegacyTemplate(templateName, settings, context.createMigrateLegacyTemplateContext()); - legacyTemplates.add(templateName); - } - } - return legacyTemplates; + @FunctionalInterface + interface TemplateCreator { + Optional createTemplate(OpenSearchClient client, String name, ObjectNode body, IClusterMetadataContext context); + } + + @FunctionalInterface + interface TemplateExistsCheck { + boolean templateAlreadyExists(OpenSearchClient client, String name); } - protected List createComponentTemplates( - GlobalMetadataData_OS_2_11 globalMetadata, - OpenSearchClient client, + + private List createTemplates( + ObjectNode templates, List templateAllowlist, + TemplateTypes templateType, + MigrationMode mode, IClusterMetadataContext context ) { - var componentTemplates = new ArrayList(); - logger.info("Setting Component Templates..."); - ObjectNode templates = globalMetadata.getComponentTemplates(); + + var templatesToCreate = new HashMap(); + var templateList = new ArrayList(); + log.info("Setting {} ...", templateType); if (templates == null) { - logger.info("No Component Templates in Snapshot"); - return componentTemplates; + log.info("No {} in Snapshot", templateType); + return List.of(); } if (templateAllowlist != null && templateAllowlist.size() == 0) { - logger.info("No Component Templates in specified allowlist"); - return componentTemplates; + log.info("No {} in specified allowlist", templateType); + return List.of(); } else if (templateAllowlist != null) { for (String templateName : templateAllowlist) { if (!templates.has(templateName) || templates.get(templateName) == null) { - logger.warn("Component Template not found: " + templateName); + log.warn("{} not found: {}", templateType, templateName); continue; } - - logger.info("Setting Component Template: " + templateName); ObjectNode settings = (ObjectNode) templates.get(templateName); - client.createComponentTemplate(templateName, settings, context.createComponentTemplateContext()); - componentTemplates.add(templateName); + templatesToCreate.put(templateName, settings); } } else { // Get the template names @@ -117,58 +137,34 @@ protected List createComponentTemplates( // Create each template for (String templateName : templateKeys) { - logger.info("Setting Component Template: " + templateName); ObjectNode settings = (ObjectNode) templates.get(templateName); - client.createComponentTemplate(templateName, settings, context.createComponentTemplateContext()); - componentTemplates.add(templateName); + templatesToCreate.put(templateName, settings); } } - return componentTemplates; - } - - protected List createIndexTemplates( - GlobalMetadataData_OS_2_11 globalMetadata, - OpenSearchClient client, - List templateAllowlist, - IClusterMetadataContext context - ) { - var indexTemplates = new ArrayList(); - logger.info("Setting Index Templates..."); - ObjectNode templates = globalMetadata.getIndexTemplates(); - - if (templates == null) { - logger.info("No Index Templates in Snapshot"); - return indexTemplates; - } - if (templateAllowlist != null && templateAllowlist.size() == 0) { - logger.info("No Index Templates in specified allowlist"); - return indexTemplates; - } else if (templateAllowlist != null) { - for (String templateName : templateAllowlist) { - if (!templates.has(templateName) || templates.get(templateName) == null) { - logger.warn("Index Template not found: " + templateName); - continue; - } - - logger.info("Setting Index Template: " + templateName); - ObjectNode settings = (ObjectNode) globalMetadata.getIndexTemplates().get(templateName); - client.createIndexTemplate(templateName, settings, context.createMigrateTemplateContext()); - indexTemplates.add(templateName); + templatesToCreate.forEach((templateName, templateBody) -> { + log.info("Creating {}: {}", templateType, templateName); + switch (mode) { + case SIMULATE: + var alreadyExists = templateType.alreadyExistsCheck.templateAlreadyExists(client, templateName); + if (!alreadyExists) { + templateList.add(templateName); + } else { + log.warn("Template {} already exists on the target, it will not be created during a migration", templateName); + } + break; + + case PERFORM: + var createdTemplate = templateType.creator.createTemplate(client, templateName, templateBody, context); + if (createdTemplate.isPresent()) { + templateList.add(templateName); + } else { + log.warn("Template {} already exists on the target, unable to create", templateName); + } + break; } - } else { - // Get the template names - List templateKeys = new ArrayList<>(); - templates.fieldNames().forEachRemaining(templateKeys::add); + }); - // Create each template - for (String templateName : templateKeys) { - logger.info("Setting Index Template: " + templateName); - ObjectNode settings = (ObjectNode) templates.get(templateName); - client.createIndexTemplate(templateName, settings, context.createMigrateTemplateContext()); - indexTemplates.add(templateName); - } - } - return indexTemplates; + return templateList; } } diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java index fe72db190..97ad3613b 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11.java @@ -1,10 +1,9 @@ package com.rfs.version_os_2_11; -import java.util.Optional; - import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.IndexCreator; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.ICreateIndexContext; @@ -20,11 +19,12 @@ public class IndexCreator_OS_2_11 implements IndexCreator { private static final ObjectMapper mapper = new ObjectMapper(); protected final OpenSearchClient client; - public Optional create( + public boolean create( IndexMetadata index, + MigrationMode mode, ICreateIndexContext context ) { - IndexMetadataData_OS_2_11 indexMetadata = new IndexMetadataData_OS_2_11(index.rawJson(), index.getId(), index.getName()); + IndexMetadataData_OS_2_11 indexMetadata = new IndexMetadataData_OS_2_11(index.getRawJson(), index.getId(), index.getName()); // Remove some settings which will cause errors if you try to pass them to the API ObjectNode settings = indexMetadata.getSettings(); @@ -42,7 +42,12 @@ public Optional create( // Create the index; it's fine if it already exists try { - return client.createIndex(index.getName(), body, context); + switch (mode) { + case SIMULATE: + return !client.hasIndex(index.getName()); + case PERFORM: + return client.createIndex(index.getName(), body, context).isPresent(); + } } catch (InvalidResponse invalidResponse) { var illegalArguments = invalidResponse.getIllegalArguments(); @@ -62,8 +67,9 @@ public Optional create( } log.info("Reattempting creation of index '" + index.getName() + "' after removing illegal arguments; " + illegalArguments); - return client.createIndex(index.getName(), body, context); + return client.createIndex(index.getName(), body, context).isPresent(); } + return false; } private void removeFieldsByPath(ObjectNode node, String path) { diff --git a/RFS/src/main/java/com/rfs/version_os_2_11/IndexMetadataData_OS_2_11.java b/RFS/src/main/java/com/rfs/version_os_2_11/IndexMetadataData_OS_2_11.java index 660bf0afa..db023e619 100644 --- a/RFS/src/main/java/com/rfs/version_os_2_11/IndexMetadataData_OS_2_11.java +++ b/RFS/src/main/java/com/rfs/version_os_2_11/IndexMetadataData_OS_2_11.java @@ -46,7 +46,7 @@ public ObjectNode getSettings() { } @Override - public ObjectNode rawJson() { + public ObjectNode getRawJson() { return root; } diff --git a/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java b/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java index e9ee759e7..2d818edeb 100644 --- a/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java +++ b/RFS/src/main/java/com/rfs/version_universal/RemoteIndexMetadata.java @@ -13,7 +13,7 @@ public class RemoteIndexMetadata implements IndexMetadata { private ObjectNode sourceData; @Override - public ObjectNode rawJson() { + public ObjectNode getRawJson() { return sourceData; } diff --git a/RFS/src/main/java/com/rfs/worker/DocumentsRunner.java b/RFS/src/main/java/com/rfs/worker/DocumentsRunner.java index 62537f813..f6168887f 100644 --- a/RFS/src/main/java/com/rfs/worker/DocumentsRunner.java +++ b/RFS/src/main/java/com/rfs/worker/DocumentsRunner.java @@ -7,8 +7,6 @@ import java.util.function.Function; import java.util.function.Supplier; -import org.apache.lucene.document.Document; - import org.opensearch.migrations.reindexer.tracing.IDocumentMigrationContexts; import com.rfs.cms.IWorkCoordinator; @@ -16,6 +14,7 @@ import com.rfs.common.DocumentReindexer; import com.rfs.common.LuceneDocumentsReader; import com.rfs.common.RfsException; +import com.rfs.common.RfsLuceneDocument; import com.rfs.common.SnapshotShardUnpacker; import com.rfs.models.ShardMetadata; import lombok.AllArgsConstructor; @@ -94,7 +93,7 @@ private void doDocumentsMigration( var unpacker = unpackerFactory.create(shardMetadata); var reader = readerFactory.apply(unpacker.unpack()); - Flux documents = reader.readDocuments(); + Flux documents = reader.readDocuments(); reindexer.reindex(shardMetadata.getIndexName(), documents, context) .doOnError(error -> log.error("Error during reindexing: " + error)) diff --git a/RFS/src/main/java/com/rfs/worker/IndexMetadataResults.java b/RFS/src/main/java/com/rfs/worker/IndexMetadataResults.java new file mode 100644 index 000000000..b63bcdbf7 --- /dev/null +++ b/RFS/src/main/java/com/rfs/worker/IndexMetadataResults.java @@ -0,0 +1,16 @@ +package com.rfs.worker; + +import java.util.List; + +import lombok.Builder; +import lombok.Data; +import lombok.Singular; + +@Builder +@Data +public class IndexMetadataResults { + @Singular + private final List indexNames; + @Singular + private final List aliases; +} diff --git a/RFS/src/main/java/com/rfs/worker/IndexRunner.java b/RFS/src/main/java/com/rfs/worker/IndexRunner.java index e8f9f73ff..7f17f8663 100644 --- a/RFS/src/main/java/com/rfs/worker/IndexRunner.java +++ b/RFS/src/main/java/com/rfs/worker/IndexRunner.java @@ -2,8 +2,8 @@ import java.util.List; import java.util.function.BiConsumer; -import java.util.stream.Collectors; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.IndexCreator; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.ICreateIndexContext; @@ -24,7 +24,7 @@ public class IndexRunner { private final Transformer transformer; private final List indexAllowlist; - public List migrateIndices(ICreateIndexContext context) { + public IndexMetadataResults migrateIndices(MigrationMode mode, ICreateIndexContext context) { SnapshotRepo.Provider repoDataProvider = metadataFactory.getRepoDataProvider(); // TODO - parallelize this, maybe ~400-1K requests per thread and do it asynchronously @@ -33,19 +33,24 @@ public List migrateIndices(ICreateIndexContext context) { log.info("Index " + indexName + " rejected by allowlist"); } }; - return repoDataProvider.getIndicesInSnapshot(snapshotName) + var results = IndexMetadataResults.builder(); + + repoDataProvider.getIndicesInSnapshot(snapshotName) .stream() .filter(FilterScheme.filterIndicesByAllowList(indexAllowlist, logger)) - .map(index -> { - var indexMetadata = metadataFactory.fromRepo(snapshotName, index.getName()); + .forEach(index -> { + var indexName = index.getName(); + var indexMetadata = metadataFactory.fromRepo(snapshotName, indexName); var transformedRoot = transformer.transformIndexMetadata(indexMetadata); - var resultOp = indexCreator.create(transformedRoot, context); - resultOp.ifPresentOrElse( - value -> log.info("Index " + index.getName() + " created successfully"), - () -> log.info("Index " + index.getName() + " already existed; no work required") - ); - return index.getName(); - }) - .collect(Collectors.toList()); + var created = indexCreator.create(transformedRoot, mode, context); + if (created) { + log.debug("Index " + indexName + " created successfully"); + results.indexName(indexName); + transformedRoot.getAliases().fieldNames().forEachRemaining(results::alias); + } else { + log.warn("Index " + indexName + " already existed; no work required"); + } + }); + return results.build(); } } diff --git a/RFS/src/main/java/com/rfs/worker/MetadataRunner.java b/RFS/src/main/java/com/rfs/worker/MetadataRunner.java index aa6a40780..b85aba34d 100644 --- a/RFS/src/main/java/com/rfs/worker/MetadataRunner.java +++ b/RFS/src/main/java/com/rfs/worker/MetadataRunner.java @@ -1,5 +1,6 @@ package com.rfs.worker; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.GlobalMetadataCreator; import org.opensearch.migrations.metadata.GlobalMetadataCreatorResults; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.IClusterMetadataContext; @@ -18,11 +19,11 @@ public class MetadataRunner { private final GlobalMetadataCreator metadataCreator; private final Transformer transformer; - public GlobalMetadataCreatorResults migrateMetadata(IClusterMetadataContext context) { + public GlobalMetadataCreatorResults migrateMetadata(MigrationMode mode, IClusterMetadataContext context) { log.info("Migrating the Templates..."); var globalMetadata = metadataFactory.fromRepo(snapshotName); var transformedRoot = transformer.transformGlobalMetadata(globalMetadata); - var results = metadataCreator.create(transformedRoot, context); + var results = metadataCreator.create(transformedRoot, mode, context); log.info("Templates migration complete"); return results; } diff --git a/RFS/src/main/java/org/opensearch/migrations/MigrationMode.java b/RFS/src/main/java/org/opensearch/migrations/MigrationMode.java new file mode 100644 index 000000000..f8b193e97 --- /dev/null +++ b/RFS/src/main/java/org/opensearch/migrations/MigrationMode.java @@ -0,0 +1,6 @@ +package org.opensearch.migrations; + +public enum MigrationMode { + SIMULATE, + PERFORM +} diff --git a/RFS/src/main/java/org/opensearch/migrations/metadata/GlobalMetadataCreator.java b/RFS/src/main/java/org/opensearch/migrations/metadata/GlobalMetadataCreator.java index eb0e614a9..f133a76f9 100644 --- a/RFS/src/main/java/org/opensearch/migrations/metadata/GlobalMetadataCreator.java +++ b/RFS/src/main/java/org/opensearch/migrations/metadata/GlobalMetadataCreator.java @@ -1,9 +1,13 @@ package org.opensearch.migrations.metadata; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.IClusterMetadataContext; import com.rfs.models.GlobalMetadata; public interface GlobalMetadataCreator { - public GlobalMetadataCreatorResults create(GlobalMetadata metadata, IClusterMetadataContext context); + public GlobalMetadataCreatorResults create( + GlobalMetadata metadata, + MigrationMode mode, + IClusterMetadataContext context); } diff --git a/RFS/src/main/java/org/opensearch/migrations/metadata/IndexCreator.java b/RFS/src/main/java/org/opensearch/migrations/metadata/IndexCreator.java index d8499ce04..47219593e 100644 --- a/RFS/src/main/java/org/opensearch/migrations/metadata/IndexCreator.java +++ b/RFS/src/main/java/org/opensearch/migrations/metadata/IndexCreator.java @@ -1,16 +1,14 @@ package org.opensearch.migrations.metadata; -import java.util.Optional; - -import com.fasterxml.jackson.databind.node.ObjectNode; - +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.ICreateIndexContext; import com.rfs.models.IndexMetadata; public interface IndexCreator { - public Optional create( + public boolean create( IndexMetadata index, + MigrationMode mode, ICreateIndexContext context ); } diff --git a/RFS/src/test/java/com/rfs/cms/WorkCoordinatorErrantAcquisitonsRetryTest.java b/RFS/src/test/java/com/rfs/cms/WorkCoordinatorErrantAcquisitonsRetryTest.java index 1b95c3ddc..4938a9049 100644 --- a/RFS/src/test/java/com/rfs/cms/WorkCoordinatorErrantAcquisitonsRetryTest.java +++ b/RFS/src/test/java/com/rfs/cms/WorkCoordinatorErrantAcquisitonsRetryTest.java @@ -13,7 +13,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; import org.opensearch.migrations.tracing.InMemoryInstrumentationBundle; @@ -108,7 +108,7 @@ private static Stream makeArgs() { @MethodSource(value = "makeArgs") public void testSecondPhaseLeaseAcquisitionFailureKeepsRetrying( Class exceptionClassToTest, - Function> responseFactory) + Function> responseFactory) throws Exception { var pathToCounts = new PathCounts(); @@ -183,23 +183,23 @@ void reset() { } @NonNull - private static Function> + private static Function> getCountingResponseMakerWithSearchBody(String searchResponse) { var payloadBytes = searchResponse.getBytes(StandardCharsets.UTF_8); return pathCounts -> getCountingResponseMaker(pathCounts, makeResponse(200, "OK", payloadBytes)); } @NonNull - private static Function> + private static Function> getCountingResponseMaker(SimpleHttpResponse searchResponse) { return pathCounts -> getCountingResponseMaker(pathCounts, searchResponse); } @NonNull - private static Function + private static Function getCountingResponseMaker(PathCounts pathToCountMap, SimpleHttpResponse searchResponse) { return httpRequestFirstLine -> { - final var uriPath = httpRequestFirstLine.path().getPath(); + final var uriPath = httpRequestFirstLine.getPath().getPath(); if (uriPath.startsWith("/" + OpenSearchWorkCoordinator.INDEX_NAME + "/_refresh")) { ++pathToCountMap.refreshes; return makeResponse(200, "OK", diff --git a/RFS/src/test/java/com/rfs/cms/WorkCoordinatorTest.java b/RFS/src/test/java/com/rfs/cms/WorkCoordinatorTest.java index 0492f9ac5..42f5c8581 100644 --- a/RFS/src/test/java/com/rfs/cms/WorkCoordinatorTest.java +++ b/RFS/src/test/java/com/rfs/cms/WorkCoordinatorTest.java @@ -13,6 +13,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.opensearch.migrations.tracing.InMemoryInstrumentationBundle; @@ -83,7 +84,6 @@ private JsonNode searchForExpiredDocs(long expirationEpochSeconds) { @Test public void testAcquireLeaseHasNoUnnecessaryConflicts() throws Exception { - log.error("Hello"); var testContext = WorkCoordinationTestContext.factory().withAllTracking(); final var NUM_DOCS = 100; try (var workCoordinator = new OpenSearchWorkCoordinator(httpClientSupplier.get(), 3600, "docCreatorWorker")) { @@ -117,6 +117,7 @@ public void testAcquireLeaseHasNoUnnecessaryConflicts() throws Exception { } @Test + @Tag("isolatedTest") public void testAcquireLeaseForQuery() throws Exception { var testContext = WorkCoordinationTestContext.factory().withAllTracking(); final var NUM_DOCS = 40; diff --git a/RFS/src/test/java/com/rfs/common/DocumentReindexerTest.java b/RFS/src/test/java/com/rfs/common/DocumentReindexerTest.java index cb079fb3d..4624f0ca1 100644 --- a/RFS/src/test/java/com/rfs/common/DocumentReindexerTest.java +++ b/RFS/src/test/java/com/rfs/common/DocumentReindexerTest.java @@ -5,10 +5,6 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Field; -import org.apache.lucene.document.StringField; -import org.apache.lucene.util.BytesRef; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -56,7 +52,7 @@ void setUp() { @Test void reindex_shouldBufferByDocumentCount() { - Flux documentStream = Flux.range(1, 10) + Flux documentStream = Flux.range(1, 10) .map(i -> createTestDocument(String.valueOf(i))); when(mockClient.sendBulkRequest(eq("test-index"), any(), any())) @@ -92,7 +88,7 @@ void reindex_shouldBufferByDocumentCount() { @Test void reindex_shouldBufferBySize() { int numDocs = 5; - Flux documentStream = Flux.range(1, numDocs) + Flux documentStream = Flux.range(1, numDocs) .map(i -> createLargeTestDocument(String.valueOf(i), MAX_BULK_SIZE / 2 + 1)); when(mockClient.sendBulkRequest(eq("test-index"), any(), any())) @@ -122,7 +118,7 @@ void reindex_shouldBufferBySize() { @Test void reindex_shouldSendDocumentsLargerThanMaxBulkSize() { - Flux documentStream = Flux.just(createLargeTestDocument("1", MAX_BULK_SIZE * 3 / 2)); + Flux documentStream = Flux.just(createLargeTestDocument("1", MAX_BULK_SIZE * 3 / 2)); when(mockClient.sendBulkRequest(eq("test-index"), any(), any())) .thenAnswer(invocation -> { @@ -148,7 +144,7 @@ void reindex_shouldSendDocumentsLargerThanMaxBulkSize() { @Test void reindex_shouldTrimAndRemoveNewlineFromSource() { - Flux documentStream = Flux.just(createTestDocumenWithWhitespace("1")); + Flux documentStream = Flux.just(createTestDocumenWithWhitespace("MQAA")); when(mockClient.sendBulkRequest(eq("test-index"), any(), any())) .thenAnswer(invocation -> { @@ -172,26 +168,17 @@ void reindex_shouldTrimAndRemoveNewlineFromSource() { assertEquals("{\"index\":{\"_id\":\"MQAA\"}}\n{\"field\":\"value\"}", capturedBulkRequests.get(0).asBulkIndex()); } - private Document createTestDocument(String id) { - Document doc = new Document(); - doc.add(new StringField("_id", new BytesRef(id), Field.Store.YES)); - doc.add(new StringField("_source", new BytesRef("{\"field\":\"value\"}"), Field.Store.YES)); - return doc; + private RfsLuceneDocument createTestDocument(String id) { + return new RfsLuceneDocument(id, "{\"field\":\"value\"}"); } - private Document createTestDocumenWithWhitespace(String id) { - Document doc = new Document(); - doc.add(new StringField("_id", new BytesRef(id), Field.Store.YES)); - doc.add(new StringField("_source", new BytesRef(" \r\n\t{\"field\"\n:\"value\"}\r\n\t "), Field.Store.YES)); - return doc; + private RfsLuceneDocument createTestDocumenWithWhitespace(String id) { + return new RfsLuceneDocument(id, " \r\n\t{\"field\"\n:\"value\"}\r\n\t "); } - private Document createLargeTestDocument(String id, int size) { - Document doc = new Document(); - doc.add(new StringField("_id", new BytesRef(id), Field.Store.YES)); + private RfsLuceneDocument createLargeTestDocument(String id, int size) { String largeField = "x".repeat(size); - doc.add(new StringField("_source", new BytesRef("{\"field\":\"" + largeField + "\"}"), Field.Store.YES)); - return doc; + return new RfsLuceneDocument(id, "{\"field\":\"" + largeField + "\"}"); } @Test @@ -200,7 +187,7 @@ void reindex_shouldRespectMaxConcurrentRequests() { int maxConcurrentRequests = 5; DocumentReindexer concurrentReindexer = new DocumentReindexer(mockClient, 1, MAX_BULK_SIZE, maxConcurrentRequests); - Flux documentStream = Flux.range(1, numDocs).map(i -> createTestDocument(String.valueOf(i))); + Flux documentStream = Flux.range(1, numDocs).map(i -> createTestDocument(String.valueOf(i))); AtomicInteger concurrentRequests = new AtomicInteger(0); AtomicInteger maxObservedConcurrency = new AtomicInteger(0); diff --git a/RFS/src/test/java/com/rfs/common/LuceneDocumentsReaderTest.java b/RFS/src/test/java/com/rfs/common/LuceneDocumentsReaderTest.java index 99286b27e..6e0297cbc 100644 --- a/RFS/src/test/java/com/rfs/common/LuceneDocumentsReaderTest.java +++ b/RFS/src/test/java/com/rfs/common/LuceneDocumentsReaderTest.java @@ -25,6 +25,7 @@ import org.apache.lucene.util.BytesRef; 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.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -97,42 +98,94 @@ public void ReadDocuments_AsExpected(Snapshot snapshot, Version version) throws Path luceneDir = unpacker.unpack(); // Use the LuceneDocumentsReader to get the documents - Flux documents = new LuceneDocumentsReader( - luceneDir, - sourceResourceProvider.getSoftDeletesPossible(), - sourceResourceProvider.getSoftDeletesFieldData() - ).readDocuments() - .sort(Comparator.comparing(doc -> Uid.decodeId(doc.getBinaryValue("_id").bytes))); // Sort for consistent order given LuceneDocumentsReader may interleave + var reader = LuceneDocumentsReader.getFactory(sourceResourceProvider).apply(luceneDir); + + Flux documents = reader.readDocuments() + .sort(Comparator.comparing(doc -> doc.id)); // Sort for consistent order given LuceneDocumentsReader may interleave // Verify that the results are as expected StepVerifier.create(documents).expectNextMatches(doc -> { String expectedId = "complexdoc"; - String actualId = Uid.decodeId(doc.getBinaryValue("_id").bytes); + String actualId = doc.id; String expectedSource = "{\"title\":\"This is a doc with complex history\",\"content\":\"Updated!\"}"; - String actualSource = doc.getBinaryValue("_source").utf8ToString(); + String actualSource = doc.source; assertDocsEqual(expectedId, actualId, expectedSource, actualSource); return true; }).expectNextMatches(doc -> { String expectedId = "unchangeddoc"; - String actualId = Uid.decodeId(doc.getBinaryValue("_id").bytes); + String actualId = doc.id; String expectedSource = "{\"title\":\"This doc will not be changed\\nIt has multiple lines of text\\nIts source doc has extra newlines.\",\"content\":\"bluh bluh\"}"; - String actualSource = doc.getBinaryValue("_source").utf8ToString(); + String actualSource = doc.source; assertDocsEqual(expectedId, actualId, expectedSource, actualSource); return true; }).expectNextMatches(doc -> { String expectedId = "updateddoc"; - String actualId = Uid.decodeId(doc.getBinaryValue("_id").bytes); + String actualId = doc.id; String expectedSource = "{\"title\":\"This is doc that will be updated\",\"content\":\"Updated!\"}"; - String actualSource = doc.getBinaryValue("_source").utf8ToString(); + String actualSource = doc.source; + assertDocsEqual(expectedId, actualId, expectedSource, actualSource); + return true; + }).expectComplete().verify(); + } + + @Test + public void ReadDocuments_ES5_Origin_AsExpected() throws Exception { + Snapshot snapshot = TestResources.SNAPSHOT_ES_6_8_MERGED; + Version version = Version.fromString("ES 6.8"); + + final var repo = new FileSystemRepo(snapshot.dir); + var sourceResourceProvider = ClusterProviderRegistry.getSnapshotReader(version, repo); + DefaultSourceRepoAccessor repoAccessor = new DefaultSourceRepoAccessor(repo); + + final ShardMetadata shardMetadata = sourceResourceProvider.getShardMetadata().fromRepo(snapshot.name, "test_updates_deletes", 0); + + SnapshotShardUnpacker unpacker = new SnapshotShardUnpacker( + repoAccessor, + tempDirectory, + shardMetadata, + Integer.MAX_VALUE + ); + Path luceneDir = unpacker.unpack(); + + // Use the LuceneDocumentsReader to get the documents + var reader = LuceneDocumentsReader.getFactory(sourceResourceProvider).apply(luceneDir); + + Flux documents = reader.readDocuments() + .sort(Comparator.comparing(doc -> doc.id)); // Sort for consistent order given LuceneDocumentsReader may interleave + + // Verify that the results are as expected + StepVerifier.create(documents).expectNextMatches(doc -> { + String expectedId = "type1#complexdoc"; + String actualId = doc.id; + + String expectedSource = "{\"title\":\"This is a doc with complex history. Updated!\"}"; + String actualSource = doc.source; + assertDocsEqual(expectedId, actualId, expectedSource, actualSource); + return true; + }).expectNextMatches(doc -> { + String expectedId = "type2#unchangeddoc"; + String actualId = doc.id; + + String expectedSource = "{\"content\":\"This doc will not be changed\nIt has multiple lines of text\nIts source doc has extra newlines.\"}"; + String actualSource = doc.source; + assertDocsEqual(expectedId, actualId, expectedSource, actualSource); + return true; + }).expectNextMatches(doc -> { + String expectedId = "type2#updateddoc"; + String actualId = doc.id; + + String expectedSource = "{\"content\":\"Updated!\"}"; + String actualSource = doc.source; assertDocsEqual(expectedId, actualId, expectedSource, actualSource); return true; }).expectComplete().verify(); } @Test + @Tag("isolatedTest") void testParallelReading() throws Exception { // Create a mock IndexReader with multiple leaves (segments) int numSegments = 10; @@ -188,7 +241,7 @@ protected DirectoryReader getReader() { }, 500, TimeUnit.MILLISECONDS); // Read documents - List actualDocuments = reader.readDocuments() + List actualDocuments = reader.readDocuments() .subscribeOn(Schedulers.parallel()) .collectList() .block(Duration.ofSeconds(2)); @@ -206,8 +259,12 @@ protected DirectoryReader getReader() { protected void assertDocsEqual(String expectedId, String actualId, String expectedSource, String actualSource) { try { - JsonNode expectedNode = objectMapper.readTree(expectedSource); - JsonNode actualNode = objectMapper.readTree(actualSource); + String sanitizedExpected = expectedSource.trim().replace("\n", "").replace("\\n", ""); + String sanitizedActual = actualSource.trim().replace("\n", "").replace("\\n", ""); + + + JsonNode expectedNode = objectMapper.readTree(sanitizedExpected); + JsonNode actualNode = objectMapper.readTree(sanitizedActual); assertEquals(expectedId, actualId); assertEquals(expectedNode, actualNode); } catch (JsonProcessingException e) { diff --git a/RFS/src/test/java/com/rfs/common/RestClientTest.java b/RFS/src/test/java/com/rfs/common/RestClientTest.java index 7b29f00ae..c76680bbf 100644 --- a/RFS/src/test/java/com/rfs/common/RestClientTest.java +++ b/RFS/src/test/java/com/rfs/common/RestClientTest.java @@ -12,7 +12,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.migrations.snapshot.creation.tracing.SnapshotTestContext; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; @@ -122,7 +122,7 @@ public void testGetEmitsInstrumentation() throws Exception { } } - SimpleHttpResponse makeResponseContext(HttpRequestFirstLine firstLine) { + SimpleHttpResponse makeResponseContext(HttpRequest firstLine) { var payloadBytes = "Hi".getBytes(StandardCharsets.UTF_8); return new SimpleHttpResponse( Map.of("Content-Type", "text/plain", "content-length", payloadBytes.length + ""), diff --git a/RFS/src/test/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10Test.java b/RFS/src/test/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10Test.java index 4a527d510..6d3aacbbb 100644 --- a/RFS/src/test/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10Test.java +++ b/RFS/src/test/java/com/rfs/version_es_7_10/SnapshotRepoData_ES_7_10Test.java @@ -44,8 +44,8 @@ void testFromRepoFile_default() { final var result = SnapshotRepoData_ES_7_10.fromRepoFile(jsonInFile); // Verify - assertThat(result.minVersion, equalTo("7.9.0")); - assertThat(result.indices.size(), equalTo(1)); + assertThat(result.getMinVersion(), equalTo("7.9.0")); + assertThat(result.getIndices().size(), equalTo(1)); } @Test @@ -58,8 +58,8 @@ void testFromRepoFile_extraFields() { final var result = SnapshotRepoData_ES_7_10.fromRepoFile(jsonInFile); // Verify - assertThat(result.minVersion, equalTo("7.9.0")); - assertThat(result.indices.size(), equalTo(1)); + assertThat(result.getMinVersion(), equalTo("7.9.0")); + assertThat(result.getIndices().size(), equalTo(1)); } private String insertAtLine(final String source, final String toAdd, final int lineNumber) { diff --git a/RFS/src/test/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11Test.java b/RFS/src/test/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11Test.java index 422dc7561..7466bfb28 100644 --- a/RFS/src/test/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11Test.java +++ b/RFS/src/test/java/com/rfs/version_os_2_11/IndexCreator_OS_2_11Test.java @@ -9,6 +9,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode; import org.junit.jupiter.api.Test; +import org.opensearch.migrations.MigrationMode; import org.opensearch.migrations.metadata.tracing.IMetadataMigrationContexts.ICreateIndexContext; import com.rfs.common.InvalidResponse; @@ -31,8 +32,7 @@ class IndexCreator_OS_2_11Test { private static final ObjectMapper OBJECT_MAPPER = JsonMapper.builder() .enable(StreamReadFeature.INCLUDE_SOURCE_IN_LOCATION) .build(); - @SuppressWarnings("unchecked") - private static final Optional INDEX_CREATE_SUCCESS = mock(Optional.class); + private static final Optional INDEX_CREATE_SUCCESS = Optional.of(mock(ObjectNode.class)); private static final String MIN_INDEX_JSON = "{ \"settings\": { } }"; @Test @@ -45,7 +45,7 @@ void testCreate() throws Exception { var result = create(client, MIN_INDEX_JSON, "indexName"); // Assertions - assertThat(result, equalTo(INDEX_CREATE_SUCCESS)); + assertThat(result, equalTo(true)); verify(client).createIndex(any(), any(), any()); } @@ -113,7 +113,7 @@ void testCreate_withRetryToRemoveValues() throws Exception { var result = create(client, rawJson, "indexName"); // Assertions - assertThat(result, equalTo(INDEX_CREATE_SUCCESS)); + assertThat(result, equalTo(true)); var requestBodyCapture = ArgumentCaptor.forClass(ObjectNode.class); verify(client, times(2)).createIndex(any(), requestBodyCapture.capture(), any()); @@ -127,11 +127,11 @@ void testCreate_withRetryToRemoveValues() throws Exception { } @SneakyThrows - private Optional create(OpenSearchClient client, String rawJson, String indexName) { + private boolean create(OpenSearchClient client, String rawJson, String indexName) { var node = (ObjectNode) OBJECT_MAPPER.readTree(rawJson); var indexId = "indexId"; - var indexData = new IndexMetadataData_OS_2_11(node, indexName, indexId); + var indexData = new IndexMetadataData_OS_2_11(node, indexId, indexName); var indexCreator = new IndexCreator_OS_2_11(client); - return indexCreator.create(indexData, mock(ICreateIndexContext.class)); + return indexCreator.create(indexData, MigrationMode.PERFORM, mock(ICreateIndexContext.class)); } } diff --git a/RFS/src/test/resources/log4j2.properties b/RFS/src/test/resources/log4j2.properties index b9b9653ce..ea5f68cc2 100644 --- a/RFS/src/test/resources/log4j2.properties +++ b/RFS/src/test/resources/log4j2.properties @@ -1,6 +1,8 @@ # Set the status level for the configuration status = DEBUG +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-DEBUG} + # Define the root logger rootLogger.level = info rootLogger.appenderRef.console.ref = Console @@ -9,11 +11,13 @@ rootLogger.appenderRef.console.ref = Console appender.console.type = Console appender.console.name = Console appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{HH:mm:ss.SSS} %threadName %-5p %c{1}:%L - %m%n +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n -# Logger definitions +# Allow customization of owned package logs logger.rfs.name = com.rfs -logger.rfs.level = debug +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} logger.wire.name = org.apache.hc.client5.http logger.wire.level = info @@ -33,4 +37,4 @@ logger.WorkCoordinator.appenderRef.stdout.ref = WorkCoordinator appender.WorkCoordinator.type = Console appender.WorkCoordinator.name = WorkCoordinator appender.WorkCoordinator.layout.type = PatternLayout -appender.WorkCoordinator.layout.pattern = %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} [worker=%X{workerId}]- %msg%n \ No newline at end of file +appender.WorkCoordinator.layout.pattern = %d{HH:mm:ss.SSS} [%t] %-5level %logger{36} [worker=%X{workerId}]- %msg%n diff --git a/RFS/src/testFixtures/java/com/rfs/framework/SearchClusterContainer.java b/RFS/src/testFixtures/java/com/rfs/framework/SearchClusterContainer.java index 13c51cae4..69458705a 100644 --- a/RFS/src/testFixtures/java/com/rfs/framework/SearchClusterContainer.java +++ b/RFS/src/testFixtures/java/com/rfs/framework/SearchClusterContainer.java @@ -72,7 +72,7 @@ public SearchClusterContainer(final ContainerVersion version) { super(DockerImageName.parse(version.imageName)); this.withExposedPorts(9200, 9300) .withEnv(version.getInitializationType().getEnvVariables()) - .waitingFor(Wait.forHttp("/").forPort(9200).forStatusCode(200).withStartupTimeout(Duration.ofMinutes(1))); + .waitingFor(Wait.forHttp("/").forPort(9200).forStatusCode(200).withStartupTimeout(Duration.ofMinutes(5))); this.containerVersion = version; } @@ -130,7 +130,7 @@ public static class ContainerVersion { public ContainerVersion(final String imageName, final Version version, INITIALIZATION_FLAVOR initializationType) { this.imageName = imageName; - this.version = version; + this.version = version; this.initializationType = initializationType; } diff --git a/RFS/src/testFixtures/java/com/rfs/http/ClusterOperations.java b/RFS/src/testFixtures/java/com/rfs/http/ClusterOperations.java index ebb7addd3..3ccd6a4c4 100644 --- a/RFS/src/testFixtures/java/com/rfs/http/ClusterOperations.java +++ b/RFS/src/testFixtures/java/com/rfs/http/ClusterOperations.java @@ -6,6 +6,7 @@ import org.apache.hc.client5.http.classic.methods.HttpDelete; import org.apache.hc.client5.http.classic.methods.HttpGet; +import org.apache.hc.client5.http.classic.methods.HttpPost; import org.apache.hc.client5.http.classic.methods.HttpPut; import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; import org.apache.hc.client5.http.impl.classic.HttpClients; @@ -170,6 +171,9 @@ public void createES7Templates( + " \"type\": \"text\"" + " }" + " }" + + " }," + + " \"aliases\": {" + + " \"alias1\": {}" + " }" + "}," + "\"version\": 1" @@ -212,4 +216,30 @@ public void createES7Templates( ); } } + + @SneakyThrows + public void createAlias(String aliasName, String indexPattern) { + final var requestBodyJson = "{\r\n" + // + " \"actions\": [\r\n" + // + " {\r\n" + // + " \"add\": {\r\n" + // + " \"index\": \"" + indexPattern + "\",\r\n" + // + " \"alias\": \"" + aliasName + "\"\r\n" + // + " }\r\n" + // + " }\r\n" + // + " ]\r\n" + // + "}"; + + final var aliasRequest = new HttpPost(this.clusterUrl + "/_aliases"); + aliasRequest.setEntity(new StringEntity(requestBodyJson)); + aliasRequest.setHeader("Content-Type", "application/json"); + + try (var response = httpClient.execute(aliasRequest)) { + assertThat( + EntityUtils.toString(response.getEntity(), StandardCharsets.UTF_8), + response.getCode(), + equalTo(200) + ); + } + } } diff --git a/TESTING.md b/TESTING.md new file mode 100644 index 000000000..eaadb87ea --- /dev/null +++ b/TESTING.md @@ -0,0 +1,28 @@ +## Gradle Usage + +[Gradle](https://gradle.org/) is used to build this repo, including its java artifacts as well as docker images. Gradle handles dependencies between projects, compiling java code, running [JUnit](https://junit.org/junit5/) tests, and building docker images. It can also deploy a demo/test environment via docker-compose for a rapid develop experience (see [dockerSolution](./TrafficCapture/dockerSolution/README.md)). + +The Gradle application is packaged within the repository, so one can simply run [gradlew](./gradlew) from the root of the repository. `./gradlew tasks` will show the tasks available at the top-level. `.../gradlew tasks` run in any subproject directory will show specific tasks that can be run for that project. Gradle can publish a scan to `scans.gradle.com` of its logs, performance, etc. at the end of its run, which can be used to diagnose a number of issues from test failures to build performance. + +This `OpensearchMigrations` Gradle project is composed of many subprojects, defined by [settings.gradle](settings.gradle). Those projects are configured similarly in the [build.gradle](./build.gradle) file. Additional settings are defined in the [gradle.properties](./gradle.properties) file. + +## Tests and Parallelization + +Gradle is configured to run most tasks from the projects in parallel, with a special exemption for tests marked to run them in total isolation of anything else that the gradle parent process is doing. Tasks that can be run in parallel include building targets and running tests. Notice that typical dependency rules apply. The number of tasks running concurrently will be limited by the maxWorkerCount that gradle is passed or configures (which is typically the # of CPUs). + +Each project within the project has the same base test configuration. Targets include `test`, `slowTest`, `isolatedTest`, and `fullTest`, which are defined within the root `build.gradle` file. Those targets are defined via the @Tag("NAME") attribute on each test (class or method). A project's `test` tasks will be run as part of its `build` task. + +A summary of each project's target composition is as follows. Notice that `allTests` exists at the top-level project to depend upon ALL of the test tasks across all of the projects. + +| Target | Composition | Purpose | +|---|---|---| +|`slowTest`| `@Tag("longTest")`| Tests that are too slow to provide value for every build run | +|`isolatedTest`| `@Tag("isolatedTest")`| Tests that may skew more toward integration tests and may take seconds to minutes to run. While these tests may require more time, they shouldn't require exhaustive use of resources or be sensitive to other tasks running in parallel | +|`test`| all other tests NOT marked with the tags above ("longTest" or "isolatedTest")| Tests that require significant or exclusive use of resources or have sensitive performance bounds | +|`fullTest`| a task dependent upon the tasks above | Convenience Task | + +The `isolatedTest` task (for each project) will run each of the tagged tests in serial and will run the isolatedTest task itself in serial from all of the other tasks within the project. While the `isolatedTest` task isn't marked as dependent upon the other tests, it is marked to run _after_ other tests if gradle is set to run them. That eliminates the requirement that test or slowTest run BEFORE the isolatedTest target when a developer is trying to only run the isolatedTest target. Likewise, `slowTest` isn't dependent upon `test`, but those two targets may run in parallel since there aren't isolation requirements. Parallelization for the test runners IS configured for `test` and `slowTest` targets so that those tests may complete quicker on hardware with more capacity. + +## Traffic Capture Memory Leak Detections + +TrafficCapture overrides the test and slowTest targets to enable netty leak detection only for slowTest. The regular test targets for the TrafficCapture subprojects sets and environment variable to disable leak detection and the `slowTest` target for those subprojects leaves the `disableMemoryLeakTests` unset but alters the tag definition to include all tests but those tagged with isolatedTest. diff --git a/TrafficCapture/README.md b/TrafficCapture/README.md index c46e3ea7b..93f4e8e89 100644 --- a/TrafficCapture/README.md +++ b/TrafficCapture/README.md @@ -154,7 +154,7 @@ This command compiles the source code and runs the quick unit tests, ensuring th For a comprehensive test run, including both quick unit tests and more extensive slow tests, execute: ```sh -../gradlew test slowTest --rerun +../gradlew allTests --rerun ``` This command initiates all tests, ensuring thorough validation of the project. The `--rerun` option is used to ignore existing task output cache for the specified tasks. diff --git a/TrafficCapture/build.gradle b/TrafficCapture/build.gradle index 7172ea770..5b3943eeb 100644 --- a/TrafficCapture/build.gradle +++ b/TrafficCapture/build.gradle @@ -33,15 +33,17 @@ allprojects { } } -tasks.named('test', Test) { - // Memory leak tests are adding too much execution time on `test` in TrafficCapture - // Disabling and will test in `slowTest` - systemProperty 'disableMemoryLeakTests', 'true' -} +subprojects { + tasks.named('test', Test) { + // Memory leak tests are adding too much execution time on `test` in TrafficCapture + // Disabling and will test in `slowTest` + it.systemProperty 'disableMemoryLeakTests', 'true' + } -tasks.named('slowTest', Test) { - useJUnitPlatform { - // Ensure rerunning all tests to run with leak detection - includeTags = [] + tasks.named('slowTest', Test) { + useJUnitPlatform { + includeTags = [] + excludeTags = ['isolatedTest'] + } } } diff --git a/TrafficCapture/captureKafkaOffloader/src/test/resources/log4j2.properties b/TrafficCapture/captureKafkaOffloader/src/test/resources/log4j2.properties index b698220a3..5e063eb26 100644 --- a/TrafficCapture/captureKafkaOffloader/src/test/resources/log4j2.properties +++ b/TrafficCapture/captureKafkaOffloader/src/test/resources/log4j2.properties @@ -1,4 +1,6 @@ -status = error +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-DEBUG} # Root logger options rootLogger.level = debug @@ -9,3 +11,9 @@ appender.console.type = Console appender.console.name = Console appender.console.layout.type = PatternLayout appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%equals{ ctx=%mdc}{ ctx=\{\}}{}%n + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/captureOffloader/src/test/resources/log4j2.properties b/TrafficCapture/captureOffloader/src/test/resources/log4j2.properties index b698220a3..5e063eb26 100644 --- a/TrafficCapture/captureOffloader/src/test/resources/log4j2.properties +++ b/TrafficCapture/captureOffloader/src/test/resources/log4j2.properties @@ -1,4 +1,6 @@ -status = error +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-DEBUG} # Root logger options rootLogger.level = debug @@ -9,3 +11,9 @@ appender.console.type = Console appender.console.name = Console appender.console.layout.type = PatternLayout appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%equals{ ctx=%mdc}{ ctx=\{\}}{}%n + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/dockerSolution/src/main/docker/elasticsearchWithSearchGuard/Dockerfile b/TrafficCapture/dockerSolution/src/main/docker/elasticsearchWithSearchGuard/Dockerfile index 2e77e4eef..9ab4c977c 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/elasticsearchWithSearchGuard/Dockerfile +++ b/TrafficCapture/dockerSolution/src/main/docker/elasticsearchWithSearchGuard/Dockerfile @@ -19,7 +19,8 @@ RUN sed 's/searchguard/plugins.security/g' $ELASTIC_SEARCH_CONFIG_FILE | \ # but maybe not for a demo to show individual steps RUN /root/enableTlsConfig.sh $ELASTIC_SEARCH_CONFIG_FILE # Alter this config line to either enable(searchguard.disabled: false) or disable(searchguard.disabled: true) HTTP auth -RUN echo "searchguard.disabled: false" >> $ELASTIC_SEARCH_CONFIG_FILE +RUN echo -n "searchguard.disabled: false" >> $ELASTIC_SEARCH_CONFIG_FILE && \ + echo -n "plugins.security.ssl.http.enabled_protocols: ['TLSv1.2', 'TLSv1.3']" >> $PROXY_TLS_CONFIG_FILE RUN sed -i '/^-Xms/i # Increase default heap to 80% RAM, Requires JDK >= 10' $ELASTIC_SEARCH_JVM_CONFIG_FILE && \ sed -i 's/^-Xms/#&/' $ELASTIC_SEARCH_JVM_CONFIG_FILE && \ diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile index c35b7de6d..d2c5b2c9e 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/Dockerfile @@ -56,7 +56,8 @@ WORKDIR /root RUN echo '. /.venv/bin/activate' >> /etc/profile.d/venv.sh RUN dnf install -y bash-completion RUN echo '. /etc/profile.d/bash_completion.sh' >> ~/.bashrc && \ - echo '. /etc/profile.d/venv.sh' >> ~/.bashrc + echo '. /etc/profile.d/venv.sh' >> ~/.bashrc && \ + echo '@echo Welcome to the Migration Assistant Console' >> ~/.bashrc # Set ENV to control startup script in /bin/sh mode ENV ENV=/root/.bashrc diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/README.md b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/README.md index 4e7c0be3d..66159b817 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/README.md +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/README.md @@ -92,6 +92,7 @@ Source and target clusters have the following options: - `endpoint`: required, the endpoint to reach the cluster. - `allow_insecure`: optional, default is false, equivalent to the curl `--insecure` flag, will not verify unsigned or invalid certificates +- `version`: optional, default is to assume a version compatible with ES 7 or OS 1. Format should be `ES_7.10.2` or `OS_2.15`, for instance. Exactly one of the following blocks must be present: @@ -198,6 +199,7 @@ The metadata migration moves indices, components, and templates from a snapshot - `index_allowlist`: optional, a list of index names. If this key is provided, only the named indices will be migrated. If the field is not provided, all non-system indices will be migrated. - `index_template_allowlist`: optional, a list of index template names. If this key is provided, only the named templates will be migrated. If the field is not provided, all templates will be migrated. - `component_template_allowlist`: optional, a list of component template names. If this key is provided, only the named component templates will be migrated. If the field is not provided, all component templates will be migrated. +- `source_cluster_version`: optional, defaults to `ES_7.10.2`, which should work for closely related versions. Version of the source cluster from which the snapshot was taken and used for handling incompatible settings between versions. - `from_snapshot`: required. As mentioned above, `from_snapshot` is the only allowable source for a metadata migration at this point. This key must be present, but if it's value is null/empty, the snapshot details will be pulled from the top-level `snapshot` object. If a `snapshot` object does not exist, this block must be populated. - `snapshot_name`: required, as described in the Snapshot section - `s3` or `fs` block: exactly one must be present, as described in the Snapshot section diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/cli.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/cli.py index 6b256d5a2..c7f5b0d01 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/cli.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/cli.py @@ -7,6 +7,7 @@ import console_link.middleware.snapshot as snapshot_ import console_link.middleware.metadata as metadata_ import console_link.middleware.replay as replay_ +import console_link.middleware.kafka as kafka_ from console_link.models.utils import ExitCode from console_link.environment import Environment @@ -311,11 +312,23 @@ def metadata_group(ctx): ignore_unknown_options=True, help_option_names=[] )) -@click.option("--detach", is_flag=True, help="Run metadata migration in detached mode") @click.argument('extra_args', nargs=-1, type=click.UNPROCESSED) @click.pass_obj -def migrate_metadata_cmd(ctx, detach, extra_args): - exitcode, message = metadata_.migrate(ctx.env.metadata, detach, extra_args) +def migrate_metadata_cmd(ctx, extra_args): + exitcode, message = metadata_.migrate(ctx.env.metadata, extra_args) + if exitcode != ExitCode.SUCCESS: + raise click.ClickException(message) + click.echo(message) + + +@metadata_group.command(name="evaluate", context_settings=dict( + ignore_unknown_options=True, + help_option_names=[] +)) +@click.argument('extra_args', nargs=-1, type=click.UNPROCESSED) +@click.pass_obj +def evaluate_metadata_cmd(ctx, extra_args): + exitcode, message = metadata_.evaluate(ctx.env.metadata, extra_args) if exitcode != ExitCode.SUCCESS: raise click.ClickException(message) click.echo(message) @@ -385,7 +398,7 @@ def kafka_group(ctx): @click.option('--topic-name', default="logging-traffic-topic", help='Specify a topic name to create') @click.pass_obj def create_topic_cmd(ctx, topic_name): - result = ctx.env.kafka.create_topic(topic_name=topic_name) + result = kafka_.create_topic(ctx.env.kafka, topic_name=topic_name) click.echo(result.value) @@ -396,13 +409,13 @@ def create_topic_cmd(ctx, topic_name): @click.pass_obj def delete_topic_cmd(ctx, acknowledge_risk, topic_name): if acknowledge_risk: - result = ctx.env.kafka.delete_topic(topic_name=topic_name) + result = kafka_.delete_topic(ctx.env.kafka, topic_name=topic_name) click.echo(result.value) else: if click.confirm('Deleting a topic will irreversibly delete all captured traffic records stored in that ' 'topic. Are you sure you want to continue?'): click.echo(f"Performing delete topic operation on {topic_name} topic...") - result = ctx.env.kafka.delete_topic(topic_name=topic_name) + result = kafka_.delete_topic(ctx.env.kafka, topic_name=topic_name) click.echo(result.value) else: click.echo("Aborting command.") @@ -412,7 +425,7 @@ def delete_topic_cmd(ctx, acknowledge_risk, topic_name): @click.option('--group-name', default="logging-group-default", help='Specify a group name to describe') @click.pass_obj def describe_group_command(ctx, group_name): - result = ctx.env.kafka.describe_consumer_group(group_name=group_name) + result = kafka_.describe_consumer_group(ctx.env.kafka, group_name=group_name) click.echo(result.value) @@ -420,7 +433,7 @@ def describe_group_command(ctx, group_name): @click.option('--topic-name', default="logging-traffic-topic", help='Specify a topic name to describe') @click.pass_obj def describe_topic_records_cmd(ctx, topic_name): - result = ctx.env.kafka.describe_topic_records(topic_name=topic_name) + result = kafka_.describe_topic_records(ctx.env.kafka, topic_name=topic_name) click.echo(result.value) # ##################### UTILITIES ################### diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/kafka.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/kafka.py index 7d5c9755a..2edabcba1 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/kafka.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/kafka.py @@ -21,5 +21,5 @@ def describe_consumer_group(kafka: Kafka, group_name: str) -> CommandResult: def describe_topic_records(kafka: Kafka, topic_name: str) -> CommandResult: - result = kafka.delete_topic(topic_name=topic_name) + result = kafka.describe_topic_records(topic_name=topic_name) return result diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/metadata.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/metadata.py index 0b8d4022a..6233fd4cc 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/metadata.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/middleware/metadata.py @@ -2,16 +2,19 @@ from console_link.middleware.error_handler import handle_errors from console_link.models.metadata import Metadata -from console_link.models.utils import ExitCode, generate_log_file_path +from console_link.models.utils import ExitCode import logging logger = logging.getLogger(__name__) @handle_errors(service_type="metadata", on_success=lambda v: (ExitCode.SUCCESS, v)) -def migrate(metadata: Metadata, detached: bool, extra_args) -> Tuple[ExitCode, str]: +def migrate(metadata: Metadata, extra_args) -> Tuple[ExitCode, str]: logger.info("Migrating metadata") - if detached: - log_file = generate_log_file_path("metadata_migration") - logger.info(f"Running in detached mode, writing logs to {log_file}") - return metadata.migrate(detached_log=log_file if detached else None, extra_args=extra_args) + return metadata.migrate(extra_args=extra_args) + + +@handle_errors(service_type="metadata", on_success=lambda v: (ExitCode.SUCCESS, v)) +def evaluate(metadata: Metadata, extra_args) -> Tuple[ExitCode, str]: + logger.info("Migrating metadata") + return metadata.evaluate(extra_args=extra_args) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/cluster.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/cluster.py index 296a31fcf..32bc46531 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/cluster.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/cluster.py @@ -60,6 +60,7 @@ "schema": { "endpoint": {"type": "string", "required": True}, "allow_insecure": {"type": "boolean", "required": False}, + "version": {"type": "string", "required": False}, "no_auth": NO_AUTH_SCHEMA, "basic_auth": BASIC_AUTH_SCHEMA, "sigv4": SIGV4_SCHEMA @@ -75,6 +76,8 @@ class Cluster: """ endpoint: str = "" + version: Optional[str] = None + aws_secret_arn: Optional[str] = None auth_type: Optional[AuthMethod] = None auth_details: Optional[Dict[str, Any]] = None allow_insecure: bool = False diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/factories.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/factories.py index a5504b8df..aa17fe96e 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/factories.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/factories.py @@ -68,7 +68,11 @@ def get_replayer(config: Dict, client_options: Optional[ClientOptions] = None): def get_kafka(config: Dict): if 'msk' in config: return MSK(config) - return StandardKafka(config) + if 'standard' in config: + return StandardKafka(config) + config.pop("broker_endpoints", None) + logger.error(f"An unsupported kafka source type was provided: {config.keys()}") + raise UnsupportedKafkaError(', '.join(config.keys())) def get_backfill(config: Dict, source_cluster: Optional[Cluster], target_cluster: Optional[Cluster], @@ -97,10 +101,8 @@ def get_backfill(config: Dict, source_cluster: Optional[Cluster], target_cluster target_cluster=target_cluster, client_options=client_options) - logger.error(f"An unsupported metrics source type was provided: {config.keys()}") - if len(config.keys()) > 1: - raise UnsupportedBackfillTypeError(', '.join(config.keys())) - raise UnsupportedBackfillTypeError(next(iter(config.keys()))) + logger.error(f"An unsupported backfill source type was provided: {config.keys()}") + raise UnsupportedBackfillTypeError(', '.join(config.keys())) def get_metrics_source(config, client_options: Optional[ClientOptions] = None): @@ -110,6 +112,4 @@ def get_metrics_source(config, client_options: Optional[ClientOptions] = None): return CloudwatchMetricsSource(config=config, client_options=client_options) else: logger.error(f"An unsupported metrics source type was provided: {config.keys()}") - if len(config.keys()) > 1: - raise UnsupportedMetricsSourceError(', '.join(config.keys())) - raise UnsupportedMetricsSourceError(next(iter(config.keys()))) + raise UnsupportedMetricsSourceError(', '.join(config.keys())) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/kafka.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/kafka.py index 38017ab6b..1791ac8e9 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/kafka.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/kafka.py @@ -29,6 +29,9 @@ } } +KAFKA_TOPICS_COMMAND = '/root/kafka-tools/kafka/bin/kafka-topics.sh' +MSK_AUTH_PARAMETERS = ['--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties'] + def get_result_for_command(command: List[str], operation_name: str) -> CommandResult: try: @@ -103,28 +106,26 @@ def __init__(self, config): super().__init__(config) def delete_topic(self, topic_name='logging-traffic-topic') -> CommandResult: - command = ['/root/kafka-tools/kafka/bin/kafka-topics.sh', '--bootstrap-server', f'{self.brokers}', '--delete', - '--topic', f'{topic_name}', '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties'] + command = [KAFKA_TOPICS_COMMAND, '--bootstrap-server', f'{self.brokers}', '--delete', + '--topic', f'{topic_name}'] + MSK_AUTH_PARAMETERS logger.info(f"Executing command: {command}") return get_result_for_command(command, "Delete Topic") def create_topic(self, topic_name='logging-traffic-topic') -> CommandResult: - command = ['/root/kafka-tools/kafka/bin/kafka-topics.sh', '--bootstrap-server', f'{self.brokers}', '--create', - '--topic', f'{topic_name}', '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties'] + command = [KAFKA_TOPICS_COMMAND, '--bootstrap-server', f'{self.brokers}', '--create', + '--topic', f'{topic_name}'] + MSK_AUTH_PARAMETERS logger.info(f"Executing command: {command}") return get_result_for_command(command, "Create Topic") def describe_consumer_group(self, group_name='logging-group-default') -> CommandResult: command = ['/root/kafka-tools/kafka/bin/kafka-consumer-groups.sh', '--bootstrap-server', f'{self.brokers}', - '--timeout', '100000', '--describe', '--group', f'{group_name}', '--command-config', - '/root/kafka-tools/aws/msk-iam-auth.properties'] + '--timeout', '100000', '--describe', '--group', f'{group_name}'] + MSK_AUTH_PARAMETERS logger.info(f"Executing command: {command}") return get_result_for_command(command, "Describe Consumer Group") def describe_topic_records(self, topic_name='logging-traffic-topic') -> CommandResult: command = ['/root/kafka-tools/kafka/bin/kafka-run-class.sh', 'kafka.tools.GetOffsetShell', '--broker-list', - f'{self.brokers}', '--topic', f'{topic_name}', '--time', '-1', '--command-config', - '/root/kafka-tools/aws/msk-iam-auth.properties'] + f'{self.brokers}', '--topic', f'{topic_name}', '--time', '-1'] + MSK_AUTH_PARAMETERS logger.info(f"Executing command: {command}") result = get_result_for_command(command, "Describe Topic Records") if result.success and result.value: @@ -142,13 +143,13 @@ def __init__(self, config): super().__init__(config) def delete_topic(self, topic_name='logging-traffic-topic') -> CommandResult: - command = ['/root/kafka-tools/kafka/bin/kafka-topics.sh', '--bootstrap-server', f'{self.brokers}', '--delete', + command = [KAFKA_TOPICS_COMMAND, '--bootstrap-server', f'{self.brokers}', '--delete', '--topic', f'{topic_name}'] logger.info(f"Executing command: {command}") return get_result_for_command(command, "Delete Topic") def create_topic(self, topic_name='logging-traffic-topic') -> CommandResult: - command = ['/root/kafka-tools/kafka/bin/kafka-topics.sh', '--bootstrap-server', f'{self.brokers}', '--create', + command = [KAFKA_TOPICS_COMMAND, '--bootstrap-server', f'{self.brokers}', '--create', '--topic', f'{topic_name}'] logger.info(f"Executing command: {command}") return get_result_for_command(command, "Create Topic") diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metadata.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metadata.py index 22faef9f3..bf4993854 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metadata.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metadata.py @@ -52,7 +52,8 @@ "min_replicas": {"type": "integer", "min": 0, "required": False}, "index_allowlist": list_schema(required=False), "index_template_allowlist": list_schema(required=False), - "component_template_allowlist": list_schema(required=False) + "component_template_allowlist": list_schema(required=False), + "source_cluster_version": {"type": "string", "required": False} } @@ -80,6 +81,7 @@ def __init__(self, config, target_cluster: Cluster, snapshot: Optional[Snapshot] self._index_template_allowlist = config.get("index_template_allowlist", None) self._component_template_allowlist = config.get("component_template_allowlist", None) self._otel_endpoint = config.get("otel_endpoint", None) + self._source_cluster_version = config.get("source_cluster_version", None) logger.debug(f"Min replicas: {self._min_replicas}") logger.debug(f"Index allowlist: {self._index_allowlist}") @@ -134,43 +136,58 @@ def _init_from_fs_snapshot(self, snapshot: FileSystemSnapshot) -> None: self._snapshot_location = "fs" self._repo_path = snapshot.repo_path - def _appendArgs(self, commands: Dict[str, Any], args_to_add: List[str]) -> None: + def _append_args(self, commands: Dict[str, Any], args_to_add: List[str]) -> None: if args_to_add is None: return - def isCommand(arg: str) -> bool: + def is_command(arg: Optional[str]) -> bool: if arg is None: return False return arg.startswith('--') or arg.startswith('-') - def isValue(arg: str) -> bool: + def is_value(arg: Optional[str]) -> bool: if arg is None: return False - return not isCommand(arg) + return not is_command(arg) i = 0 while i < len(args_to_add): arg = args_to_add[i] - nextArg = args_to_add[i + 1] if (i + 1 < len(args_to_add)) else None + next_arg = args_to_add[i + 1] if (i + 1 < len(args_to_add)) else None - if isCommand(arg) and isValue(nextArg): - commands[arg] = nextArg + if is_command(arg) and is_value(next_arg): + commands[arg] = next_arg i += 2 # Move past the command and value - elif isCommand(arg): + elif is_command(arg): commands[arg] = None i += 1 # Move past the command, its a flag else: logger.warning(f"Ignoring extra value {arg}, there was no command name before it") i += 1 - def migrate(self, detached_log=None, extra_args=None) -> CommandResult: + def evaluate(self, extra_args=None) -> CommandResult: logger.info("Starting metadata migration") + return self.migrate_or_evaluate("evaluate", extra_args) + + def migrate(self, extra_args=None) -> CommandResult: + logger.info("Starting metadata migration") + return self.migrate_or_evaluate("migrate", extra_args) + + def migrate_or_evaluate(self, command: str, extra_args=None) -> CommandResult: command_base = "/root/metadataMigration/bin/MetadataMigration" - command_args = { + command_args = {} + + # Add any common metadata parameter before the command + if self._otel_endpoint: + command_args.update({"--otel-collector-endpoint": self._otel_endpoint}) + + command_args.update({ + command: None, "--snapshot-name": self._snapshot_name, "--target-host": self._target_cluster.endpoint, "--min-replicas": self._min_replicas - } + }) + if self._snapshot_location == 's3': command_args.update({ "--s3-local-dir": self._local_dir, @@ -211,16 +228,14 @@ def migrate(self, detached_log=None, extra_args=None) -> CommandResult: if self._component_template_allowlist: command_args.update({"--component-template-allowlist": ",".join(self._component_template_allowlist)}) - if self._otel_endpoint: - command_args.update({"--otel-collector-endpoint": self._otel_endpoint}) + if self._source_cluster_version: + command_args.update({"--source-version": self._source_cluster_version}) # Extra args might not be represented with dictionary, so convert args to list and append commands - self._appendArgs(command_args, extra_args) + self._append_args(command_args, extra_args) command_runner = CommandRunner(command_base, command_args, - sensitive_fields=["--target-password"], - run_as_detatched=detached_log is not None, - log_file=detached_log) + sensitive_fields=["--target-password"]) logger.info(f"Migrating metadata with command: {' '.join(command_runner.sanitized_command())}") try: return command_runner.run() diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metrics_source.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metrics_source.py index d7fb95074..097865bb5 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metrics_source.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/console_link/models/metrics_source.py @@ -73,9 +73,9 @@ def get_metric_data( component: Component, metric: str, statistic: MetricStatistic, - startTime: datetime, + start_time: datetime, period_in_seconds: int = 60, - endTime: Optional[datetime] = None, + end_time: Optional[datetime] = None, dimensions: Optional[Dict] = None, ) -> List[Tuple[str, float]]: raise NotImplementedError @@ -145,21 +145,21 @@ def get_metric_data( component: Component, metric: str, statistic: MetricStatistic, - startTime: datetime, + start_time: datetime, period_in_seconds: int = 60, - endTime: Optional[datetime] = None, + end_time: Optional[datetime] = None, dimensions: Optional[Dict[str, str]] = None, ) -> List[Tuple[str, float]]: logger.info(f"{self.__class__.__name__}.get_metric_data called with {component=}, {metric=}, {statistic=}," - f"{startTime=}, {period_in_seconds=}, {endTime=}, {dimensions=}") + f"{start_time=}, {period_in_seconds=}, {end_time=}, {dimensions=}") aws_dimensions = [{"Name": "OTelLib", "Value": component.value}] if dimensions: aws_dimensions += [{"Name": k, "Value": v} for k, v in dimensions.items()] logger.debug(f"AWS Dimensions set to: {aws_dimensions}") - if not endTime: - endTime = datetime.now() - logger.debug(f"No endTime provided, using current time: {endTime}") + if not end_time: + end_time = datetime.now() + logger.debug(f"No endTime provided, using current time: {end_time}") response = self.client.get_metric_data( MetricDataQueries=[ { @@ -175,8 +175,8 @@ def get_metric_data( }, }, ], - StartTime=startTime, - EndTime=endTime, + StartTime=start_time, + EndTime=end_time, ScanBy="TimestampAscending", ) raise_for_aws_api_error(response) @@ -197,8 +197,7 @@ def prometheus_component_names(c: Component) -> str: return "capture" elif c == Component.REPLAYER: return "replay" - else: - raise ValueError(f"Unsupported component: {c}") + raise ValueError(f"Unsupported component: {c}") class PrometheusMetricsSource(MetricsSource): @@ -239,15 +238,15 @@ def get_metric_data( component: Component, metric: str, statistic: MetricStatistic, - startTime: datetime, + start_time: datetime, period_in_seconds: int = 60, - endTime: Optional[datetime] = None, + end_time: Optional[datetime] = None, dimensions: Optional[Dict] = None, ) -> List[Tuple[str, float]]: logger.info(f"{self.__class__.__name__} get_metric_data called with {component=}, {metric=}, {statistic=}," - f"{startTime=}, {period_in_seconds=}, {endTime=}, {dimensions=}") - if not endTime: - endTime = datetime.now() + f"{start_time=}, {period_in_seconds=}, {end_time=}, {dimensions=}") + if not end_time: + end_time = datetime.now() headers = None if self.client_options and self.client_options.user_agent_extra: headers = append_user_agent_header_for_requests(headers=None, @@ -256,8 +255,8 @@ def get_metric_data( f"{self.endpoint}/api/v1/query_range", params={ # type: ignore "query": f'{metric}{{exported_job="{prometheus_component_names(component)}"}}', - "start": startTime.timestamp(), - "end": endTime.timestamp(), + "start": start_time.timestamp(), + "end": end_time.timestamp(), "step": period_in_seconds, }, headers=headers, diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_backfill.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_backfill.py index 565c22017..8d9c712b8 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_backfill.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_backfill.py @@ -2,8 +2,10 @@ import pathlib import pytest +import requests import requests_mock +from console_link.models.cluster import Cluster from console_link.models.backfill_base import Backfill, BackfillStatus from console_link.models.backfill_osi import OpenSearchIngestionBackfill from console_link.models.backfill_rfs import DockerRFSBackfill, ECSRFSBackfill @@ -131,6 +133,17 @@ def test_get_backfill_unsupported_type(): assert "fetch" in str(excinfo.value.args[1]) +def test_get_backfill_multiple_types(): + unknown_config = { + "fetch": {"data": "xyz"}, + "new_backfill": {"data": "abc"} + } + with pytest.raises(UnsupportedBackfillTypeError) as excinfo: + get_backfill(unknown_config, None, None) + assert "fetch" in excinfo.value.args[1] + assert "new_backfill" in excinfo.value.args[1] + + def test_cant_instantiate_with_multiple_types(): config = { "opensearch_ingestion": { @@ -245,7 +258,6 @@ def test_ecs_rfs_calculates_backfill_status_from_ecs_instance_statuses_running(e assert str(mocked_running_status) == value.value[1] -@pytest.mark.skip(reason="Need to implement mocking for multiple calls to endpoint") def test_ecs_rfs_get_status_deep_check(ecs_rfs_backfill, mocker): target = create_valid_cluster() mocked_instance_status = InstanceStatuses( @@ -269,3 +281,39 @@ def test_ecs_rfs_get_status_deep_check(ecs_rfs_backfill, mocker): assert BackfillStatus.RUNNING == value.value[0] assert str(mocked_instance_status) in value.value[1] assert str(total_shards) in value.value[1] + + +def test_ecs_rfs_deep_status_check_failure(ecs_rfs_backfill, mocker, caplog): + mocked_instance_status = InstanceStatuses( + desired=1, + running=1, + pending=0 + ) + mock_ecs = mocker.patch.object(ECSService, 'get_instance_statuses', autospec=True, + return_value=mocked_instance_status) + mock_api = mocker.patch.object(Cluster, 'call_api', side_effect=requests.exceptions.RequestException()) + result = ecs_rfs_backfill.get_status(deep_check=True) + assert "Working state index does not yet exist" in caplog.text + mock_ecs.assert_called_once() + mock_api.assert_called_once() + assert result.success + assert result.value[0] == BackfillStatus.RUNNING + + +def test_docker_backfill_not_implemented_commands(): + docker_rfs_config = { + "reindex_from_snapshot": { + "docker": None + } + } + docker_rfs_backfill = get_backfill(docker_rfs_config, None, target_cluster=create_valid_cluster()) + assert isinstance(docker_rfs_backfill, DockerRFSBackfill) + + with pytest.raises(NotImplementedError): + docker_rfs_backfill.start() + + with pytest.raises(NotImplementedError): + docker_rfs_backfill.stop() + + with pytest.raises(NotImplementedError): + docker_rfs_backfill.scale(units=3) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cli.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cli.py index 3c3f64b28..6ace00371 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cli.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cli.py @@ -14,6 +14,7 @@ from console_link.models.command_result import CommandResult from console_link.models.ecs_service import ECSService, InstanceStatuses from console_link.models.kafka import StandardKafka +from console_link.models.metrics_source import Component from console_link.models.replayer_ecs import ECSReplayer TEST_DATA_DIRECTORY = pathlib.Path(__file__).parent / "data" @@ -42,6 +43,38 @@ def set_fake_aws_credentials(): os.environ['AWS_ACCESS_KEY_ID'] = 'AKIAIOSFODNN7EXAMPLE' os.environ['AWS_SECRET_ACCESS_KEY'] = 'wJalrXUtnFEMI/K7MDENG/bPxRfiCYEXAMPLEKEY' + +@pytest.fixture +def source_cluster_only_yaml_path(tmp_path): + source_cluster_only_path = tmp_path / "source_cluster_only.yaml" + source_cluster_only_yaml = """ +source_cluster: + endpoint: "https://elasticsearch:9200" + allow_insecure: true + basic_auth: + username: "admin" + password: "admin" +""" + with open(source_cluster_only_path, 'w') as f: + f.write(source_cluster_only_yaml) + return source_cluster_only_path + + +@pytest.fixture +def target_cluster_only_yaml_path(tmp_path): + target_cluster_only_path = tmp_path / "target_cluster_only.yaml" + target_cluster_only_yaml = """ +target_cluster: + endpoint: "https://opensearchtarget:9200" + allow_insecure: true + basic_auth: + username: "admin" + password: "myStrongPassword123!" +""" + with open(target_cluster_only_path, 'w') as f: + f.write(target_cluster_only_yaml) + return target_cluster_only_path + # Tests around the general CLI functionality @@ -58,12 +91,36 @@ def test_cli_with_valid_services_file_does_not_raise_error(runner): assert result.exit_code == 0 +def test_cli_with_no_clusters_in_services_raises_error(runner, tmp_path): + no_cluster_services = """ +metrics_source: + prometheus: + endpoint: "http://prometheus:9090" +backfill: + reindex_from_snapshot: + docker: +snapshot: + snapshot_name: "test_snapshot" + fs: + repo_path: "/snapshot/test-console" + otel_endpoint: "http://otel-collector:4317" + """ + yaml_path = tmp_path / "services.yaml" + with open(yaml_path, 'w') as f: + f.write(no_cluster_services) + + result = runner.invoke(cli, ['--config-file', str(yaml_path), 'clusters', 'connection-check'], + catch_exceptions=True) + assert result.exit_code == 1 + assert isinstance(result.exception, SystemExit) + # The following tests are mostly smoke-tests with a goal of covering every CLI command and option. # They generally mock functions either at the logic or the model layer, though occasionally going all the way to # an external endpoint call. # Standardizing these in the future would be great, but the priority right now is getting overall coverage, and # testing that . + def test_cli_cluster_cat_indices(runner, mocker): middleware_mock = mocker.spy(middleware.clusters, 'cat_indices') api_mock = mocker.patch.object(Cluster, 'call_api') @@ -100,6 +157,52 @@ def test_cli_cluster_connection_check(runner, mocker): api_mock.assert_called() +def test_cli_cluster_cat_indices_and_connection_check_with_one_cluster(runner, mocker, + target_cluster_only_yaml_path, + source_cluster_only_yaml_path): + middleware_connection_check_mock = mocker.spy(middleware.clusters, 'connection_check') + middleware_cat_indices_mock = mocker.spy(middleware.clusters, 'cat_indices') + api_mock = mocker.patch.object(Cluster, 'call_api', autospec=True) + # Connection check with no target cluster + result = runner.invoke(cli, ['--config-file', str(source_cluster_only_yaml_path), 'clusters', 'connection-check'], + catch_exceptions=True) + assert result.exit_code == 0 + assert "SOURCE CLUSTER" in result.output + assert "No target cluster defined." in result.output + middleware_connection_check_mock.assert_called_once() + api_mock.assert_called_once() + middleware_connection_check_mock.reset_mock() + api_mock.reset_mock() + # Connection check with no source cluster + result = runner.invoke(cli, ['--config-file', str(target_cluster_only_yaml_path), 'clusters', 'connection-check'], + catch_exceptions=True) + assert result.exit_code == 0 + assert "TARGET CLUSTER" in result.output + assert "No source cluster defined." in result.output + middleware_connection_check_mock.assert_called_once() + api_mock.assert_called_once() + middleware_connection_check_mock.reset_mock() + api_mock.reset_mock() + # Cat indices with no target cluster + result = runner.invoke(cli, ['--config-file', str(source_cluster_only_yaml_path), 'clusters', 'cat-indices'], + catch_exceptions=True) + assert result.exit_code == 0 + assert "SOURCE CLUSTER" in result.output + assert "No target cluster defined." in result.output + middleware_cat_indices_mock.assert_called_once() + api_mock.assert_called_once() + middleware_cat_indices_mock.reset_mock() + api_mock.reset_mock() + # Cat indices with no source cluster + result = runner.invoke(cli, ['--config-file', str(target_cluster_only_yaml_path), 'clusters', 'cat-indices'], + catch_exceptions=True) + assert result.exit_code == 0 + assert "TARGET CLUSTER" in result.output + assert "No source cluster defined." in result.output + middleware_cat_indices_mock.assert_called_once() + api_mock.assert_called_once() + + def test_cli_cluster_run_test_benchmarks(runner, mocker): middleware_mock = mocker.spy(middleware.clusters, 'run_test_benchmarks') model_mock = mocker.patch.object(Cluster, 'execute_benchmark_workload') @@ -110,6 +213,16 @@ def test_cli_cluster_run_test_benchmarks(runner, mocker): assert result.exit_code == 0 +def test_cli_cluster_run_test_benchmarks_without_source_raises_error(runner, mocker, target_cluster_only_yaml_path): + middleware_mock = mocker.spy(middleware.clusters, 'run_test_benchmarks') + model_mock = mocker.patch.object(Cluster, 'execute_benchmark_workload') + result = runner.invoke(cli, ['--config-file', target_cluster_only_yaml_path, 'clusters', 'run-test-benchmarks'], + catch_exceptions=True) + middleware_mock.assert_not_called() + model_mock.assert_not_called() + assert result.exit_code == 2 + + def test_cli_cluster_clear_indices(runner, mocker): mock = mocker.patch('console_link.middleware.clusters.clear_indices') result = runner.invoke(cli, @@ -160,6 +273,13 @@ def test_cli_cat_indices_e2e(runner, env): assert target_cat_indices in result.output +def test_cli_snapshot_when_not_defined_raises_error(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'snapshot', 'create'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Snapshot is not set" in result.output + + def test_cli_snapshot_create(runner, mocker): mock = mocker.patch('console_link.middleware.snapshot.create') @@ -201,6 +321,13 @@ def test_cli_with_backfill_describe(runner, mocker): assert result.exit_code == 0 +def test_cli_backfill_when_not_defined(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'backfill', 'start'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Backfill migration is not set" in result.output + + def test_cli_backfill_create_rfs(runner, mocker): mock = mocker.patch.object(ECSRFSBackfill, 'create', autospec=True) result = runner.invoke(cli, ['--config-file', str(TEST_DATA_DIRECTORY / "services_with_ecs_rfs.yaml"), @@ -293,6 +420,13 @@ def test_get_backfill_status_with_deep_check(runner, mocker): mock_detailed_status_call.assert_called_once() +def test_cli_replay_when_not_defined(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'replay', 'describe'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Replay is not set" in result.output + + def test_replay_describe(runner, mocker): mock = mocker.patch('console_link.middleware.replay.describe') result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), 'replay', 'describe'], @@ -341,6 +475,13 @@ def test_replay_status(runner, mocker): assert result.exit_code == 0 +def test_cli_metadata_when_not_defined(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'metadata', 'migrate'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Metadata is not set" in result.output + + def test_cli_metadata_migrate(runner, mocker): mock = mocker.patch("subprocess.run") result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), 'metadata', 'migrate'], @@ -349,7 +490,22 @@ def test_cli_metadata_migrate(runner, mocker): assert result.exit_code == 0 -def test_cli_with_metrics_get_data(runner, mocker): +def test_cli_metadata_evaluate(runner, mocker): + mock = mocker.patch("subprocess.run") + result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), 'metadata', 'evaluate'], + catch_exceptions=True) + mock.assert_called_once() + assert result.exit_code == 0 + + +def test_cli_metrics_when_not_defined(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'metrics', 'list'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Metrics source is not set" in result.output + + +def test_cli_with_metrics_list_metrics(runner, mocker): mock = mocker.patch('console_link.models.metrics_source.PrometheusMetricsSource.get_metrics') result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), 'metrics', 'list'], catch_exceptions=True) @@ -357,41 +513,98 @@ def test_cli_with_metrics_get_data(runner, mocker): assert result.exit_code == 0 -def test_cli_kafka_create_topic(runner, mocker): - # These commands _should_ go through the middleware layer but currently don't - # middleware_mock = mocker.spy(middleware.kafka, 'create_topic') - # middleware_mock.assert_called_once_with(env.kafka, 'test') +def test_cli_with_metrics_list_metrics_as_json(runner, mocker): + mock = mocker.patch('console_link.models.metrics_source.PrometheusMetricsSource.get_metrics', + return_value={'captureProxy': ['kafkaCommitCount', 'captureConnectionDuration'], + 'replayer': ['kafkaCommitCount']}, autospec=True) + result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), '--json', 'metrics', 'list'], + catch_exceptions=True) + mock.assert_called_once() + assert result.exit_code == 0 + +def test_cli_with_metrics_get_data(runner, mocker): + mock = mocker.patch('console_link.models.metrics_source.PrometheusMetricsSource.get_metric_data', + return_value=[('2024-05-22T20:06:00+00:00', 0.0), ('2024-05-22T20:07:00+00:00', 1.0), + ('2024-05-22T20:08:00+00:00', 2.0), ('2024-05-22T20:09:00+00:00', 3.0), + ('2024-05-22T20:10:00+00:00', 4.0)], + autospec=True) + result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), 'metrics', 'get-data', + 'replayer', 'kafkaCommitCount'], + catch_exceptions=True) + assert result.exit_code == 0 + mock.assert_called_once() + assert mock.call_args.args[1] == Component.REPLAYER + assert mock.call_args.args[2] == 'kafkaCommitCount' + + +def test_cli_with_metrics_get_data_as_json(runner, mocker): + mock = mocker.patch('console_link.models.metrics_source.PrometheusMetricsSource.get_metric_data', + return_value=[('2024-05-22T20:06:00+00:00', 0.0), ('2024-05-22T20:07:00+00:00', 1.0), + ('2024-05-22T20:08:00+00:00', 2.0), ('2024-05-22T20:09:00+00:00', 3.0), + ('2024-05-22T20:10:00+00:00', 4.0)], + autospec=True) + result = runner.invoke(cli, ['--config-file', str(VALID_SERVICES_YAML), '--json', 'metrics', 'get-data', + 'replayer', 'kafkaCommitCount'], + catch_exceptions=True) + assert result.exit_code == 0 + mock.assert_called_once() + assert mock.call_args.args[1] == Component.REPLAYER + assert mock.call_args.args[2] == 'kafkaCommitCount' + + +def test_cli_kafka_when_not_defined(runner, source_cluster_only_yaml_path): + result = runner.invoke(cli, ['--config-file', source_cluster_only_yaml_path, 'kafka', 'create-topic'], + catch_exceptions=True) + assert result.exit_code == 2 + assert "Kafka is not set" in result.output + + +def test_cli_kafka_create_topic(runner, mocker): + middleware_mock = mocker.spy(middleware.kafka, 'create_topic') model_mock = mocker.patch.object(StandardKafka, 'create_topic') result = runner.invoke(cli, ['-vv', '--config-file', str(VALID_SERVICES_YAML), 'kafka', 'create-topic', '--topic-name', 'test'], catch_exceptions=True) + model_mock.assert_called_once_with(topic_name='test') + middleware_mock.assert_called_once() assert result.exit_code == 0 def test_cli_kafka_delete_topic(runner, mocker): model_mock = mocker.patch.object(StandardKafka, 'delete_topic') + middleware_mock = mocker.spy(middleware.kafka, 'delete_topic') result = runner.invoke(cli, ['-vv', '--config-file', str(VALID_SERVICES_YAML), 'kafka', 'delete-topic', '--topic-name', 'test', '--acknowledge-risk'], catch_exceptions=True) model_mock.assert_called_once_with(topic_name='test') + middleware_mock.assert_called_once() assert result.exit_code == 0 def test_cli_kafka_describe_consumer_group(runner, mocker): model_mock = mocker.patch.object(StandardKafka, 'describe_consumer_group') + middleware_mock = mocker.spy(middleware.kafka, 'describe_consumer_group') result = runner.invoke(cli, ['-vv', '--config-file', str(VALID_SERVICES_YAML), 'kafka', 'describe-consumer-group', '--group-name', 'test-group'], catch_exceptions=True) model_mock.assert_called_once_with(group_name='test-group') + middleware_mock.assert_called_once() assert result.exit_code == 0 def test_cli_kafka_describe_topic(runner, mocker): model_mock = mocker.patch.object(StandardKafka, 'describe_topic_records') + middleware_mock = mocker.spy(middleware.kafka, 'describe_topic_records') result = runner.invoke(cli, ['-vv', '--config-file', str(VALID_SERVICES_YAML), 'kafka', 'describe-topic-records', '--topic-name', 'test'], catch_exceptions=True) model_mock.assert_called_once_with(topic_name='test') + middleware_mock.assert_called_once() + assert result.exit_code == 0 + + +def test_completion_script(runner): + result = runner.invoke(cli, [str(VALID_SERVICES_YAML), 'completion', 'bash'], catch_exceptions=True) assert result.exit_code == 0 diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cluster.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cluster.py index ebbc03946..1d7bfb7cf 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cluster.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_cluster.py @@ -305,3 +305,65 @@ def test_valid_cluster_api_call_with_sigv4_auth(requests_mock, aws_credentials): assert "Signature=" in auth_header assert "es" in auth_header assert "us-east-2" in auth_header + + +def test_call_api_via_middleware(requests_mock): + cluster = create_valid_cluster(auth_type=AuthMethod.NO_AUTH) + requests_mock.get(f"{cluster.endpoint}/test_api", json={'test': True}) + + response = clusters_.call_api(cluster, '/test_api') + assert response.status_code == 200 + assert response.json() == {'test': True} + + +def test_cat_indices_with_refresh(requests_mock): + cluster = create_valid_cluster(auth_type=AuthMethod.NO_AUTH) + refresh_mock = requests_mock.get(f"{cluster.endpoint}/_refresh") + indices_mock = requests_mock.get(f"{cluster.endpoint}/_cat/indices/_all") + + clusters_.cat_indices(cluster, refresh=True) + assert refresh_mock.call_count == 1 + assert indices_mock.call_count == 1 + + +def test_clear_indices(requests_mock): + cluster = create_valid_cluster(auth_type=AuthMethod.NO_AUTH) + mock = requests_mock.delete(f"{cluster.endpoint}/*,-.*,-searchguard*,-sg7*,.migrations_working_state") + clusters_.clear_indices(cluster) + assert mock.call_count == 1 + + +def test_run_benchmark_executes_correctly_no_auth(mocker): + cluster = create_valid_cluster(auth_type=AuthMethod.NO_AUTH) + mock = mocker.patch("subprocess.run", autospec=True) + workload = "nyctaxis" + cluster.execute_benchmark_workload(workload=workload) + mock.assert_called_once_with("opensearch-benchmark execute-test --distribution-version=1.0.0 " + f"--target-host={cluster.endpoint} --workload={workload} --pipeline=benchmark-only" + " --test-mode --kill-running-processes --workload-params=target_throughput:0.5," + "bulk_size:10,bulk_indexing_clients:1,search_clients:1 " + "--client-options=verify_certs:false", shell=True) + + +def test_run_benchmark_raises_error_sigv4_auth(): + cluster = create_valid_cluster(auth_type=AuthMethod.SIGV4, details={"region": "eu-west-1", "service": "aoss"}) + workload = "nyctaxis" + with pytest.raises(NotImplementedError): + cluster.execute_benchmark_workload(workload=workload) + + +def test_run_benchmark_executes_correctly_basic_auth_and_https(mocker): + auth_details = {"username": "admin", "password": "Admin1"} + cluster = create_valid_cluster(auth_type=AuthMethod.BASIC_AUTH, details=auth_details) + cluster.allow_insecure = False + + mock = mocker.patch("subprocess.run", autospec=True) + workload = "nyctaxis" + cluster.execute_benchmark_workload(workload=workload) + mock.assert_called_once_with("opensearch-benchmark execute-test --distribution-version=1.0.0 " + f"--target-host={cluster.endpoint} --workload={workload} --pipeline=benchmark-only" + " --test-mode --kill-running-processes --workload-params=target_throughput:0.5," + "bulk_size:10,bulk_indexing_clients:1,search_clients:1 " + "--client-options=verify_certs:false,use_ssl:true," + f"basic_auth_user:{auth_details['username']}," + f"basic_auth_password:{auth_details['password']}", shell=True) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_kafka.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_kafka.py new file mode 100644 index 000000000..2290fda23 --- /dev/null +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_kafka.py @@ -0,0 +1,194 @@ +import pytest + +from console_link.models.factories import UnsupportedKafkaError, get_kafka +from console_link.models.kafka import Kafka, MSK, StandardKafka + + +def test_get_msk_kafka(): + config = { + "broker_endpoints": "abc", + "msk": None + } + kafka = get_kafka(config) + assert isinstance(kafka, Kafka) + assert isinstance(kafka, MSK) + + +def test_get_standard_kafka(): + config = { + "broker_endpoints": "abc", + "standard": None + } + kafka = get_kafka(config) + assert isinstance(kafka, Kafka) + assert isinstance(kafka, StandardKafka) + + +def test_unsupported_kafka_type_raises_error(): + config = { + "broker_endpoints": "abc", + "new_kafka_type": None + } + with pytest.raises(UnsupportedKafkaError) as exc_info: + get_kafka(config) + assert 'new_kafka_type' in exc_info.value.args + + +def test_no_kafka_type_raises_error(): + config = { + "broker_endpoints": "abc", + } + with pytest.raises(UnsupportedKafkaError): + get_kafka(config) + + +def test_multiple_kafka_types_raises_error(): + config = { + "broker_endpoints": "abc", + "msk": None, + "standard": None + } + with pytest.raises(ValueError) as exc_info: + get_kafka(config) + + assert "More than one value is present" in exc_info.value.args[0]['kafka'][0] + + +def test_msk_kafka_create_topic(mocker): + config = { + "broker_endpoints": "abc", + "msk": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.create_topic(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-topics.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--create', + '--topic', 'new_topic', '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties' + ], capture_output=True, text=True, check=True) + + +def test_standard_kafka_create_topic(mocker): + config = { + "broker_endpoints": "abc", + "standard": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.create_topic(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-topics.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--create', + '--topic', 'new_topic' + ], capture_output=True, text=True, check=True) + + +def test_msk_kafka_delete_topic(mocker): + config = { + "broker_endpoints": "abc", + "msk": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.delete_topic(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-topics.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--delete', + '--topic', 'new_topic', '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties' + ], capture_output=True, text=True, check=True) + + +def test_standard_kafka_delete_topic(mocker): + config = { + "broker_endpoints": "abc", + "standard": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.delete_topic(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-topics.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--delete', + '--topic', 'new_topic' + ], capture_output=True, text=True, check=True) + + +def test_msk_kafka_describe_topic(mocker): + config = { + "broker_endpoints": "abc", + "msk": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.describe_topic_records(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-run-class.sh', 'kafka.tools.GetOffsetShell', + '--broker-list', f"{config['broker_endpoints']}", + '--topic', 'new_topic', + '--time', '-1', + '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties' + ], capture_output=True, text=True, check=True) + + +def test_standard_kafka_describe_topic(mocker): + config = { + "broker_endpoints": "abc", + "standard": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.describe_topic_records(topic_name='new_topic') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-run-class.sh', 'kafka.tools.GetOffsetShell', + '--broker-list', f"{config['broker_endpoints']}", + '--topic', 'new_topic', + '--time', '-1' + ], capture_output=True, text=True, check=True) + + +def test_msk_kafka_describe_group(mocker): + config = { + "broker_endpoints": "abc", + "msk": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.describe_consumer_group(group_name='new_group') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-consumer-groups.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--timeout', '100000', '--describe', + '--group', 'new_group', + '--command-config', '/root/kafka-tools/aws/msk-iam-auth.properties' + ], capture_output=True, text=True, check=True) + + +def test_standard_kafka_describe_group(mocker): + config = { + "broker_endpoints": "abc", + "standard": None + } + kafka = get_kafka(config) + mock = mocker.patch('subprocess.run', autospec=True) + result = kafka.describe_consumer_group(group_name='new_group') + + assert result.success + mock.assert_called_once_with( + ['/root/kafka-tools/kafka/bin/kafka-consumer-groups.sh', + '--bootstrap-server', f"{config['broker_endpoints']}", '--timeout', '100000', '--describe', + '--group', 'new_group', + ], capture_output=True, text=True, check=True) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metadata.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metadata.py index fe8db19af..23e797610 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metadata.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metadata.py @@ -141,6 +141,30 @@ def test_full_config_and_snapshot_gives_priority_to_config(s3_snapshot): assert metadata._local_dir == config["from_snapshot"]["local_dir"] +def test_full_config_with_version_includes_version_string_in_subprocess(s3_snapshot, mocker): + config = { + "from_snapshot": { + "local_dir": "/tmp/s3", + "snapshot_name": "reindex_from_snapshot", + "s3": { + "repo_uri": "s3://my-bucket", + "aws_region": "us-east-1" + }, + }, + "source_cluster_version": "ES_6.8" + + } + metadata = Metadata(config, create_valid_cluster(), s3_snapshot) + + mock = mocker.patch("subprocess.run") + metadata.migrate() + + mock.assert_called_once() + actual_call_args = mock.call_args.args[0] + assert '--source-version' in actual_call_args + assert config['source_cluster_version'] in actual_call_args + + def test_metadata_with_s3_snapshot_makes_correct_subprocess_call(mocker): config = { "from_snapshot": { @@ -161,6 +185,8 @@ def test_metadata_with_s3_snapshot_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "--otel-collector-endpoint", config["otel_endpoint"], + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '0', @@ -168,7 +194,6 @@ def test_metadata_with_s3_snapshot_makes_correct_subprocess_call(mocker): "--s3-repo-uri", config["from_snapshot"]["s3"]["repo_uri"], "--s3-region", config["from_snapshot"]["s3"]["aws_region"], "--target-insecure", - "--otel-collector-endpoint", config["otel_endpoint"], ], stdout=None, stderr=None, text=True, check=True ) @@ -191,12 +216,13 @@ def test_metadata_with_fs_snapshot_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "--otel-collector-endpoint", config["otel_endpoint"], + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '0', "--file-system-repo-path", config["from_snapshot"]["fs"]["repo_path"], "--target-insecure", - "--otel-collector-endpoint", config["otel_endpoint"], ], stdout=None, stderr=None, text=True, check=True) @@ -218,6 +244,7 @@ def test_metadata_with_min_replicas_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '2', @@ -248,6 +275,8 @@ def test_metadata_with_allowlists_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "--otel-collector-endpoint", config["otel_endpoint"], + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '0', @@ -256,30 +285,10 @@ def test_metadata_with_allowlists_makes_correct_subprocess_call(mocker): "--index-allowlist", "index1,index2", "--index-template-allowlist", "index_template1,index_template2", "--component-template-allowlist", "component_template1,component_template2", - "--otel-collector-endpoint", config["otel_endpoint"], ], stdout=None, stderr=None, text=True, check=True ) -def test_metadata_migrate_detached_makes_correct_subprocess_call(mocker): - config = { - "from_snapshot": { - "snapshot_name": "reindex_from_snapshot", - "fs": { - "repo_path": "path/to/repo" - }, - }, - "min_replicas": 2, - } - target = create_valid_cluster(auth_type=AuthMethod.NO_AUTH) - metadata = Metadata(config, target, None) - - mock = mocker.patch("subprocess.Popen") - metadata.migrate(detached_log="/tmp/log_file.log") - - mock.assert_called_once() - - def test_metadata_with_target_config_auth_makes_correct_subprocess_call(mocker): config = { "from_snapshot": { @@ -299,6 +308,7 @@ def test_metadata_with_target_config_auth_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '0', @@ -334,6 +344,7 @@ def test_metadata_with_target_sigv4_makes_correct_subprocess_call(mocker): mock.assert_called_once_with([ "/root/metadataMigration/bin/MetadataMigration", + "migrate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], "--target-host", target.endpoint, "--min-replicas", '0', @@ -360,7 +371,7 @@ def test_metadata_init_with_minimal_config_and_extra_args(mocker): metadata = Metadata(config, create_valid_cluster(), None) mock = mocker.patch("subprocess.run") - metadata.migrate(extra_args=[ + metadata.evaluate(extra_args=[ "--foo", "bar", # Pair of command and value "--flag", # Flag with no value afterward "--bar", "baz", # Another pair of command and value @@ -371,6 +382,7 @@ def test_metadata_init_with_minimal_config_and_extra_args(mocker): mock.assert_called_once_with([ '/root/metadataMigration/bin/MetadataMigration', + "evaluate", "--snapshot-name", config["from_snapshot"]["snapshot_name"], '--target-host', 'https://opensearchtarget:9200', '--min-replicas', '0', diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metrics_source.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metrics_source.py index 417c20102..d353f6e11 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metrics_source.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_metrics_source.py @@ -151,7 +151,7 @@ def test_cloudwatch_metrics_get_metric_data(cw_ms, cw_stubber): cw_ms.client = cw_stubber.client metrics = cw_ms.get_metric_data(Component.CAPTUREPROXY, "kafkaCommitCount", - MetricStatistic.Average, startTime=datetime.datetime.now()) + MetricStatistic.Average, start_time=datetime.datetime.now()) assert metrics == mock_metric_data @@ -184,5 +184,5 @@ def test_prometheus_get_metric_for_nonexistent_component(prometheus_ms): with pytest.raises(ValueError): prometheus_ms.get_metric_data( Component(3), "kafkaCommitCount", - MetricStatistic.Average, startTime=datetime.datetime.now() + MetricStatistic.Average, start_time=datetime.datetime.now() ) diff --git a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_replay.py b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_replay.py index bfa374fe5..9d534af76 100644 --- a/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_replay.py +++ b/TrafficCapture/dockerSolution/src/main/docker/migrationConsole/lib/console_link/tests/test_replay.py @@ -6,9 +6,10 @@ import console_link.middleware.replay as replay_ from console_link.models.ecs_service import ECSService -from console_link.models.factories import get_replayer +from console_link.models.factories import UnsupportedReplayerError, get_replayer from console_link.models.replayer_base import Replayer from console_link.models.replayer_ecs import ECSReplayer +from console_link.models.replayer_docker import DockerReplayer TEST_DATA_DIRECTORY = pathlib.Path(__file__).parent / "data" AWS_REGION = "us-east-1" @@ -116,3 +117,22 @@ def test_replayer_describe_as_json(): success, output = replay_.describe(replayer, as_json=True) assert success assert json.loads(output) == config + + +def test_get_docker_replayer(): + config = { + "docker": None + } + replayer = get_replayer(config) + assert isinstance(replayer, DockerReplayer) + + +def test_nonexistent_replayer_type(): + config = { + "new_replayer_type": { + "setting": "value" + } + } + with pytest.raises(UnsupportedReplayerError) as exc_info: + get_replayer(config) + assert 'new_replayer_type' in exc_info.value.args diff --git a/TrafficCapture/gradle.properties b/TrafficCapture/gradle.properties deleted file mode 100644 index 5ca1b8ea9..000000000 --- a/TrafficCapture/gradle.properties +++ /dev/null @@ -1,7 +0,0 @@ -org.gradle.caching=true -org.gradle.configuration-cache=true -org.gradle.configureondemand=true - -# Set Gradle Daemon's idle timeout to 30 minutes -org.gradle.daemon.idletimeout=1800000 -org.gradle.parallel=true \ No newline at end of file diff --git a/TrafficCapture/nettyWireLogging/src/test/resources/log4j2.properties b/TrafficCapture/nettyWireLogging/src/test/resources/log4j2.properties index 0636de958..79f28d0bb 100644 --- a/TrafficCapture/nettyWireLogging/src/test/resources/log4j2.properties +++ b/TrafficCapture/nettyWireLogging/src/test/resources/log4j2.properties @@ -1,5 +1,8 @@ -status = error +status = WARN +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-DEBUG} + +# Root logger options rootLogger.level = debug appender.console.type = Console @@ -8,4 +11,8 @@ appender.console.target = SYSTEM_ERR appender.console.layout.type = PatternLayout appender.console.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} [%t] %c{1} - %msg%equals{ ctx=%mdc}{ ctx=\{\}}{}%n -rootLogger.appenderRef.stderr.ref = STDERR +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/trafficCaptureProxyServer/build.gradle b/TrafficCapture/trafficCaptureProxyServer/build.gradle index 2f0732c09..bb3430705 100644 --- a/TrafficCapture/trafficCaptureProxyServer/build.gradle +++ b/TrafficCapture/trafficCaptureProxyServer/build.gradle @@ -32,15 +32,17 @@ dependencies { implementation project(':TrafficCapture:captureKafkaOffloader') implementation project(':coreUtilities') + implementation group: "com.google.protobuf", name: "protobuf-java" + implementation group: 'com.fasterxml.jackson.core', name: 'jackson-core' + implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind' + implementation group: 'com.lmax', name: 'disruptor' implementation group: 'io.netty', name: 'netty-all' implementation group: 'org.apache.logging.log4j', name: 'log4j-api' implementation group: 'org.apache.logging.log4j', name: 'log4j-core' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'org.slf4j', name: 'slf4j-api' - implementation group: 'com.lmax', name: 'disruptor' - implementation group: 'com.beust', name: 'jcommander' - implementation group: "com.google.protobuf", name: "protobuf-java" testImplementation project(':TrafficCapture:captureProtobufs') testImplementation testFixtures(project(path: ':testHelperFixtures')) diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java index bbb350147..4d4981246 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxy.java @@ -1,26 +1,32 @@ package org.opensearch.migrations.trafficcapture.proxyserver; +import java.io.File; import java.io.FileReader; import java.io.IOException; import java.io.OutputStream; import java.net.URI; -import java.nio.file.Files; +import java.nio.charset.StandardCharsets; import java.nio.file.Paths; import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Properties; -import java.util.TreeMap; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import java.util.stream.Collectors; import java.util.stream.Stream; import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLException; import com.google.protobuf.CodedOutputStream; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -38,8 +44,12 @@ import org.opensearch.migrations.trafficcapture.StreamLifecycleManager; import org.opensearch.migrations.trafficcapture.kafkaoffloader.KafkaCaptureFactory; import org.opensearch.migrations.trafficcapture.netty.HeaderValueFilteringCapturePredicate; +import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; import org.opensearch.migrations.trafficcapture.proxyserver.netty.BacksideConnectionPool; +import org.opensearch.migrations.trafficcapture.proxyserver.netty.HeaderAdderHandler; +import org.opensearch.migrations.trafficcapture.proxyserver.netty.HeaderRemoverHandler; import org.opensearch.migrations.trafficcapture.proxyserver.netty.NettyScanningHttpProxy; +import org.opensearch.migrations.trafficcapture.proxyserver.netty.ProxyChannelInitializer; import org.opensearch.migrations.utils.ProcessHelpers; import org.opensearch.security.ssl.DefaultSecurityKeyStore; import org.opensearch.security.ssl.util.SSLConfigConstants; @@ -47,6 +57,9 @@ import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; import com.beust.jcommander.ParameterException; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.socket.SocketChannel; import io.netty.handler.ssl.SslContext; import io.netty.handler.ssl.SslContextBuilder; import io.netty.handler.ssl.util.InsecureTrustManagerFactory; @@ -60,61 +73,101 @@ public class CaptureProxy { private static final String HTTPS_CONFIG_PREFIX = "plugins.security.ssl.http."; public static final String DEFAULT_KAFKA_CLIENT_ID = "HttpCaptureProxyProducer"; + public static final String SUPPORTED_TLS_PROTOCOLS_LIST_KEY = "plugins.security.ssl.http.enabled_protocols"; public static class Parameters { - @Parameter(required = false, names = { - "--traceDirectory" }, arity = 1, description = "Directory to store trace files in.") + @Parameter(required = false, + names = { "--traceDirectory" }, + arity = 1, + description = "Directory to store trace files in.") public String traceDirectory; - @Parameter(required = false, names = { - "--noCapture" }, arity = 0, description = "If enabled, Does NOT capture traffic to ANY sink.") + @Parameter(required = false, + names = { "--noCapture" }, + arity = 0, + description = "If enabled, Does NOT capture traffic to ANY sink.") public boolean noCapture; - @Parameter(required = false, names = { - "--kafkaConfigFile" }, arity = 1, description = "Kafka properties file for additional client customization.") + @Parameter(required = false, + names = { "--kafkaConfigFile" }, + arity = 1, + description = "Kafka properties file for additional client customization.") public String kafkaPropertiesFile; - @Parameter(required = false, names = { - "--kafkaClientId" }, arity = 1, description = "clientId to use for interfacing with Kafka.") + @Parameter(required = false, + names = { "--kafkaClientId" }, + arity = 1, + description = "clientId to use for interfacing with Kafka.") public String kafkaClientId = DEFAULT_KAFKA_CLIENT_ID; - @Parameter(required = false, names = { - "--kafkaConnection" }, arity = 1, description = "Sequence of values delimited by ','.") + @Parameter(required = false, + names = { "--kafkaConnection" }, + arity = 1, + description = "Sequence of values delimited by ','.") public String kafkaConnection; - @Parameter(required = false, names = { - "--enableMSKAuth" }, arity = 0, description = "Enables SASL Kafka properties required for connecting to MSK with IAM auth.") + @Parameter(required = false, + names = { "--enableMSKAuth" }, + arity = 0, + description = "Enables SASL Kafka properties required for connecting to MSK with IAM auth.") public boolean mskAuthEnabled = false; - @Parameter(required = false, names = { - "--sslConfigFile" }, arity = 1, description = "YAML configuration of the HTTPS settings. When this is not set, the proxy will not use TLS.") + @Parameter(required = false, + names = { "--sslConfigFile" }, + arity = 1, + description = "YAML configuration of the HTTPS settings. When this is not set, the proxy will not use TLS.") public String sslConfigFilePath; - @Parameter(required = false, names = { - "--maxTrafficBufferSize" }, arity = 1, description = "The maximum number of bytes that will be written to a single TrafficStream object.") + @Parameter(required = false, + names = { "--maxTrafficBufferSize" }, + arity = 1, + description = "The maximum number of bytes that will be written to a single TrafficStream object.") public int maximumTrafficStreamSize = 1024 * 1024; - @Parameter(required = false, names = { - "--insecureDestination" }, arity = 0, description = "Do not check the destination server's certificate") + @Parameter(required = false, + names = { "--insecureDestination" }, + arity = 0, + description = "Do not check the destination server's certificate") public boolean allowInsecureConnectionsToBackside; - @Parameter(required = true, names = { - "--destinationUri" }, arity = 1, description = "URI of the server that the proxy is capturing traffic for.") + @Parameter(required = true, + names = { "--destinationUri" }, + arity = 1, + description = "URI of the server that the proxy is capturing traffic for.") public String backsideUriString; - @Parameter(required = true, names = { - "--listenPort" }, arity = 1, description = "Exposed port for clients to connect to this proxy.") + @Parameter(required = true, + names = { "--listenPort" }, + arity = 1, + description = "Exposed port for clients to connect to this proxy.") public int frontsidePort = 0; - @Parameter(required = false, names = { - "--numThreads" }, arity = 1, description = "How many threads netty should create in its event loop group") + @Parameter(required = false, + names = { "--numThreads" }, + arity = 1, + description = "How many threads netty should create in its event loop group") public int numThreads = 1; - @Parameter(required = false, names = { - "--destinationConnectionPoolSize" }, arity = 1, description = "Number of socket connections that should be maintained to the destination server " + @Parameter(required = false, + names = { "--destinationConnectionPoolSize" }, + arity = 1, + description = "Number of socket connections that should be maintained to the destination server " + "to reduce the perceived latency to clients. Each thread will have its own cache, so the " + "total number of outstanding warm connections will be multiplied by numThreads.") public int destinationConnectionPoolSize = 0; - @Parameter(required = false, names = { - "--destinationConnectionPoolTimeout" }, arity = 1, description = "Of the socket connections maintained by the destination connection pool, " + @Parameter(required = false, + names = { "--destinationConnectionPoolTimeout" }, + arity = 1, + description = "Of the socket connections maintained by the destination connection pool, " + "how long after connection should the be recycled " + "(closed with a new connection taking its place)") public String destinationConnectionPoolTimeout = "PT30S"; - @Parameter(required = false, names = { - "--otelCollectorEndpoint" }, arity = 1, description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be forwarded." + @Parameter(required = false, + names = { "--otelCollectorEndpoint" }, + arity = 1, + description = "Endpoint (host:port) for the OpenTelemetry Collector to which metrics logs should be forwarded." + "If this is not provided, metrics will not be sent to a collector.") public String otelCollectorEndpoint; - @Parameter(required = false, names = "--suppressCaptureForHeaderMatch", arity = 2, description = "The header name (which will be interpreted in a case-insensitive manner) and a regex " - + "pattern. When the incoming request has a header that matches the regex, it will be passed " - + "through to the service but will NOT be captured. E.g. user-agent 'healthcheck'.") + @Parameter(required = false, + names = "--setHeader", + arity = 2, + description = "[header-name header-value] Set an HTTP header (first argument) with to the specified value" + + " (second argument). Any existing headers with that name will be removed.") + public List headerOverrides = new ArrayList<>(); + @Parameter(required = false, + names = "--suppressCaptureForHeaderMatch", + arity = 2, + description = "The header name (which will be interpreted in a case-insensitive manner) and a regex " + + "pattern. When the incoming request has a header that matches the regex, it will be passed " + + "through to the service but will NOT be captured. E.g. user-agent 'healthcheck'.") public List suppressCaptureHeaderPairs = new ArrayList<>(); } @@ -143,19 +196,26 @@ static Parameters parseArgs(String[] args) { @SneakyThrows protected static Settings getSettings(@NonNull String configFile) { - var builder = Settings.builder(); - try (var lines = Files.lines(Paths.get(configFile))) { - lines.map( - line -> Optional.of(line.indexOf('#')).filter(i -> i >= 0).map(i -> line.substring(0, i)).orElse(line) - ).filter(line -> line.startsWith(HTTPS_CONFIG_PREFIX) && line.contains(":")).forEach(line -> { - var parts = line.split(": *", 2); - builder.put(parts[0], parts[1]); - }); - } - builder.put(SSLConfigConstants.SECURITY_SSL_TRANSPORT_ENABLED, false); + var objectMapper = new ObjectMapper(new YAMLFactory()); + var configMap = objectMapper.readValue(new File(configFile), Map.class); + var configParentDirStr = Paths.get(configFile).toAbsolutePath().getParent(); - builder.put("path.home", configParentDirStr); - return builder.build(); + var httpsSettings = + objectMapper.convertValue(configMap, new TypeReference>(){}) + .entrySet().stream() + .filter(kvp -> kvp.getKey().startsWith(HTTPS_CONFIG_PREFIX)) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (!httpsSettings.containsKey(SUPPORTED_TLS_PROTOCOLS_LIST_KEY)) { + httpsSettings.put(SUPPORTED_TLS_PROTOCOLS_LIST_KEY, List.of("TLSv1.2", "TLSv1.3")); + } + + return Settings.builder().loadFromMap(httpsSettings) + // Don't bother with configurations the 'transport' (port 9300), which the plugin that we're using + // will also configure (& fail) otherwise. We only use the plugin to setup security for the 'http' + // port and then move the SSLEngine into our implementation. + .put(SSLConfigConstants.SECURITY_SSL_TRANSPORT_ENABLED, false) + .put("path.home", configParentDirStr) + .build(); } protected static IConnectionCaptureFactory getNullConnectionCaptureFactory() { @@ -299,7 +359,10 @@ protected static SslContext loadBacksideSslContext(URI serverUri, boolean allowI } protected static Map convertPairListToMap(List list) { - var map = new TreeMap(); + if (list == null) { + return Map.of(); + } + var map = new LinkedHashMap(); for (int i = 0; i < list.size(); i += 2) { map.put(list.get(i), list.get(i + 1)); } @@ -313,19 +376,17 @@ public static void main(String[] args) throws InterruptedException, IOException var params = parseArgs(args); var backsideUri = convertStringToUri(params.backsideUriString); - var rootContext = new RootCaptureContext( + var ctx = new RootCaptureContext( RootOtelContext.initializeOpenTelemetryWithCollectorOrAsNoop(params.otelCollectorEndpoint, "capture", ProcessHelpers.getNodeInstanceName()), new CompositeContextTracker(new ActiveContextTracker(), new ActiveContextTrackerByActivityType()) ); var sksOp = Optional.ofNullable(params.sslConfigFilePath) - .map( - sslConfigFile -> new DefaultSecurityKeyStore( - getSettings(sslConfigFile), - Paths.get(sslConfigFile).toAbsolutePath().getParent() - ) - ); + .map(sslConfigFile -> new DefaultSecurityKeyStore( + getSettings(sslConfigFile), + Paths.get(sslConfigFile).toAbsolutePath().getParent())) + .filter(sks -> sks.sslHTTPProvider != null); sksOp.ifPresent(DefaultSecurityKeyStore::initHttpSSLConfig); var proxy = new NettyScanningHttpProxy(params.frontsidePort); @@ -349,14 +410,10 @@ public static void main(String[] args) throws InterruptedException, IOException var headerCapturePredicate = new HeaderValueFilteringCapturePredicate( convertPairListToMap(params.suppressCaptureHeaderPairs) ); - proxy.start( - rootContext, - backsideConnectionPool, - params.numThreads, - sslEngineSupplier, - getConnectionCaptureFactory(params, rootContext), - headerCapturePredicate - ); + var proxyChannelInitializer = + buildProxyChannelInitializer(ctx, backsideConnectionPool, sslEngineSupplier, headerCapturePredicate, + params.headerOverrides, getConnectionCaptureFactory(params, ctx)); + proxy.start(proxyChannelInitializer, params.numThreads); } catch (Exception e) { log.atError().setCause(e).setMessage("Caught exception while setting up the server and rethrowing").log(); throw e; @@ -375,4 +432,53 @@ public static void main(String[] args) throws InterruptedException, IOException // work in the background. proxy.waitForClose(); } + + static ProxyChannelInitializer buildProxyChannelInitializer(RootCaptureContext rootContext, + BacksideConnectionPool backsideConnectionPool, + Supplier sslEngineSupplier, + @NonNull RequestCapturePredicate headerCapturePredicate, + List headerOverridesArgs, + IConnectionCaptureFactory connectionFactory) + { + var headers = new ArrayList<>(convertPairListToMap(headerOverridesArgs).entrySet()); + Collections.reverse(headers); + final var removeStrings = new ArrayList(headers.size()); + final var addBufs = new ArrayList(headers.size()); + + for (var kvp : headers) { + addBufs.add( + Unpooled.unreleasableBuffer( + Unpooled.wrappedBuffer( + (kvp.getKey() + ": " + kvp.getValue()).getBytes(StandardCharsets.UTF_8)))); + removeStrings.add(kvp.getKey() + ":"); + } + + return new ProxyChannelInitializer( + rootContext, + backsideConnectionPool, + sslEngineSupplier, + connectionFactory, + headerCapturePredicate + ) { + @Override + protected void initChannel(@NonNull SocketChannel ch) throws IOException { + super.initChannel(ch); + final var pipeline = ch.pipeline(); + { + int i = 0; + for (var kvp : headers) { + pipeline.addAfter(ProxyChannelInitializer.CAPTURE_HANDLER_NAME, "AddHeader-" + kvp.getKey(), + new HeaderAdderHandler(addBufs.get(i++))); + } + } + { + int i = 0; + for (var kvp : headers) { + pipeline.addAfter(ProxyChannelInitializer.CAPTURE_HANDLER_NAME, "RemoveHeader-" + kvp.getKey(), + new HeaderRemoverHandler(removeStrings.get(i++))); + } + } + } + }; + } } diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandler.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandler.java new file mode 100644 index 000000000..1eefb08a1 --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandler.java @@ -0,0 +1,60 @@ +package org.opensearch.migrations.trafficcapture.proxyserver.netty; + +import java.nio.charset.StandardCharsets; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.ReferenceCountUtil; + +public class HeaderAdderHandler extends ChannelInboundHandlerAdapter { + private static final ByteBuf CRLF_BYTE_BUF = + Unpooled.unreleasableBuffer(Unpooled.wrappedBuffer("\r\n".getBytes(StandardCharsets.UTF_8))); + private static final ByteBuf LF_BYTE_BUF = + Unpooled.unreleasableBuffer(Unpooled.wrappedBuffer("\n".getBytes(StandardCharsets.UTF_8))); + boolean insertedHeader = false; + private final ByteBuf headerLineToAdd; + boolean useCarriageReturn; + + public HeaderAdderHandler(ByteBuf headerLineToAdd) { + this.headerLineToAdd = headerLineToAdd.retain(); + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (!(msg instanceof ByteBuf) || insertedHeader) { + super.channelRead(ctx, msg); + return; + } + var buf = (ByteBuf) msg; + buf.markReaderIndex(); + while (buf.isReadable()) { + var nextByte = buf.readByte(); + if (nextByte == '\r') { + useCarriageReturn = true; + } else if (nextByte == '\n') { + final var upToIndex = buf.readerIndex(); + var composite = ctx.alloc().compositeBuffer(4); + buf.resetReaderIndex(); + final var startingIndex = buf.readerIndex(); + composite.addComponent(true, buf.retainedSlice(startingIndex, upToIndex-startingIndex)); + composite.addComponent(true, headerLineToAdd.retainedDuplicate()); + composite.addComponent(true, (useCarriageReturn ? CRLF_BYTE_BUF : LF_BYTE_BUF).duplicate()); + composite.addComponent(true, buf.retainedSlice(upToIndex, buf.readableBytes()-upToIndex)); + buf.release(); + super.channelRead(ctx, composite); + insertedHeader = true; + return; + } + } + buf.resetReaderIndex(); + super.channelRead(ctx, msg); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + ReferenceCountUtil.release(headerLineToAdd); + super.channelUnregistered(ctx); + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandler.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandler.java new file mode 100644 index 000000000..ec981cce1 --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandler.java @@ -0,0 +1,169 @@ +package org.opensearch.migrations.trafficcapture.proxyserver.netty; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.CompositeByteBuf; +import io.netty.channel.ChannelHandlerContext; +import io.netty.channel.ChannelInboundHandlerAdapter; +import io.netty.util.ReferenceCountUtil; +import lombok.SneakyThrows; +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class HeaderRemoverHandler extends ChannelInboundHandlerAdapter { + final String headerToRemove; + CompositeByteBuf previousRemaining; + // This handler has 3 states - copying, dropping, or testing. when previousRemaining != null, we're testing. + // when dropUntilNewline == true, we're dropping, otherwise, we're copying (when previousRemaining==null) + // The starting state is previousRemaining == null and dropUntilNewline = false + boolean dropUntilNewline; + MessagePosition requestPosition = MessagePosition.IN_HEADER; + + private enum MessagePosition { + IN_HEADER, ONE_NEW_LINE, AFTER_HEADERS, + } + + public HeaderRemoverHandler(String headerToRemove) { + if (!headerToRemove.endsWith(":")) { + throw new IllegalArgumentException("The headerToRemove must end with a ':'"); + } + this.headerToRemove = headerToRemove; + } + + @SneakyThrows + void lambdaSafeSuperChannelRead(ChannelHandlerContext ctx, ByteBuf bb) { + super.channelRead(ctx, bb); + } + + /** + * @return true if there's a discongruity in the incoming buf and the contents that preceded this call will + * need to be buffered by the caller + */ + boolean matchNextBytes(ChannelHandlerContext ctx, ByteBuf buf) { + final var sourceReaderIdx = buf.readerIndex(); + for (int i=previousRemaining.writerIndex(); ; ++i) { + if (!buf.isReadable()) { // partial match + previousRemaining.addComponent(true, + buf.retainedSlice(sourceReaderIdx, i-previousRemaining.writerIndex())); + return true; + } + if (i == headerToRemove.length()) { // match! + previousRemaining.release(); // drop those in limbo ... + previousRemaining = null; + dropUntilNewline = true; // ... plus other bytes until we reset + return true; + } + buf.markReaderIndex(); + if (Character.toLowerCase(headerToRemove.charAt(i)) != Character.toLowerCase(buf.readByte())) { // no match + flushAndClearPreviousRemaining(ctx); + buf.resetReaderIndex(); + dropUntilNewline = false; + return false; + } + } + } + + void flushAndClearPreviousRemaining(ChannelHandlerContext ctx) { + previousRemaining.forEach(bb -> lambdaSafeSuperChannelRead(ctx, bb.retain())); + previousRemaining.removeComponents(0, previousRemaining.numComponents()); + previousRemaining.release(); + previousRemaining = null; + } + + boolean advanceByteBufUntilNewline(ByteBuf bb) { + while (bb.isReadable()) { // sonar lint doesn't like if the while statement has an empty body + if (bb.readByte() == '\n') { return true; } + } + return false; + } + + ByteBuf addSliceToRunningBuf(ChannelHandlerContext ctx, ByteBuf priorBuf, ByteBuf sourceBuf, + int start, int len) { + if (len == 0) { + return priorBuf; + } + var slicedSourceBuf = sourceBuf.retainedSlice(start, len); + if (priorBuf == null) { + return slicedSourceBuf; + } + CompositeByteBuf cbb; + if (!(priorBuf instanceof CompositeByteBuf)) { + cbb = ctx.alloc().compositeBuffer(4); + cbb.addComponent(true, priorBuf); + } else { + cbb = (CompositeByteBuf) priorBuf; + } + cbb.addComponent(true, slicedSourceBuf); + return cbb; + } + + @Override + public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception { + if (!(msg instanceof ByteBuf) || requestPosition == MessagePosition.AFTER_HEADERS) { + super.channelRead(ctx, msg); + return; + } + + var sourceBuf = (ByteBuf) msg; + var currentSourceSegmentStart = + (previousRemaining != null || dropUntilNewline || requestPosition == MessagePosition.ONE_NEW_LINE) + ? -1 : sourceBuf.readerIndex(); + ByteBuf cleanedIncomingBuf = null; + sourceBuf.markReaderIndex(); + + while (sourceBuf.isReadable()) { + if (requestPosition == MessagePosition.ONE_NEW_LINE) { + final var nextByte = sourceBuf.readByte(); + if (nextByte == '\n' || nextByte == '\r') { + requestPosition = MessagePosition.AFTER_HEADERS; + if (currentSourceSegmentStart == -1) { + currentSourceSegmentStart = sourceBuf.readerIndex() - 1; + } + sourceBuf.readerIndex(sourceBuf.writerIndex()); + break; + } else { + previousRemaining = ctx.alloc().compositeBuffer(16); + requestPosition = MessagePosition.IN_HEADER; + sourceBuf.resetReaderIndex(); + continue; + } + } + + if (previousRemaining != null) { + final var sourceReaderIdx = sourceBuf.readerIndex(); + if (matchNextBytes(ctx, sourceBuf)) { + if (currentSourceSegmentStart >= 0 && + sourceReaderIdx != currentSourceSegmentStart) // would be 0-length + { + cleanedIncomingBuf = addSliceToRunningBuf(ctx, cleanedIncomingBuf, sourceBuf, + currentSourceSegmentStart, sourceReaderIdx-currentSourceSegmentStart); + currentSourceSegmentStart = -1; + } + } else if (currentSourceSegmentStart == -1) { + currentSourceSegmentStart = sourceReaderIdx; + } + } else { + if (advanceByteBufUntilNewline(sourceBuf)) { + sourceBuf.markReaderIndex(); + requestPosition = MessagePosition.ONE_NEW_LINE; + } else { + break; + } + } + } + + if (currentSourceSegmentStart >= 0) { + cleanedIncomingBuf = addSliceToRunningBuf(ctx, cleanedIncomingBuf, sourceBuf, + currentSourceSegmentStart, sourceBuf.readerIndex()-currentSourceSegmentStart); + } + sourceBuf.release(); + if (cleanedIncomingBuf != null) { + super.channelRead(ctx, cleanedIncomingBuf); + } + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + ReferenceCountUtil.release(previousRemaining); + super.channelUnregistered(ctx); + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java index f8770930b..9ed135890 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxy.java @@ -1,11 +1,6 @@ package org.opensearch.migrations.trafficcapture.proxyserver.netty; -import java.util.function.Supplier; -import javax.net.ssl.SSLEngine; -import org.opensearch.migrations.trafficcapture.IConnectionCaptureFactory; -import org.opensearch.migrations.trafficcapture.netty.RequestCapturePredicate; -import org.opensearch.migrations.trafficcapture.netty.tracing.IRootWireLoggingContext; import io.netty.bootstrap.ServerBootstrap; import io.netty.channel.Channel; @@ -14,9 +9,10 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.nio.NioServerSocketChannel; import io.netty.util.concurrent.DefaultThreadFactory; -import lombok.NonNull; +import lombok.Getter; public class NettyScanningHttpProxy { + @Getter protected final int proxyPort; protected Channel mainChannel; protected EventLoopGroup workerGroup; @@ -26,33 +22,16 @@ public NettyScanningHttpProxy(int proxyPort) { this.proxyPort = proxyPort; } - public int getProxyPort() { - return proxyPort; - } - - public void start( - IRootWireLoggingContext rootContext, - BacksideConnectionPool backsideConnectionPool, - int numThreads, - Supplier sslEngineSupplier, - IConnectionCaptureFactory connectionCaptureFactory, - @NonNull RequestCapturePredicate requestCapturePredicate - ) throws InterruptedException { + public void start(ProxyChannelInitializer proxyChannelInitializer, int numThreads) + throws InterruptedException + { bossGroup = new NioEventLoopGroup(1, new DefaultThreadFactory("captureProxyPoolBoss")); workerGroup = new NioEventLoopGroup(numThreads, new DefaultThreadFactory("captureProxyPoolWorker")); ServerBootstrap serverBootstrap = new ServerBootstrap(); try { mainChannel = serverBootstrap.group(bossGroup, workerGroup) .channel(NioServerSocketChannel.class) - .childHandler( - new ProxyChannelInitializer<>( - rootContext, - backsideConnectionPool, - sslEngineSupplier, - connectionCaptureFactory, - requestCapturePredicate - ) - ) + .childHandler(proxyChannelInitializer) .childOption(ChannelOption.AUTO_READ, false) .bind(proxyPort) .sync() diff --git a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java index b32a6e6e8..96972ec60 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/main/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ProxyChannelInitializer.java @@ -17,6 +17,8 @@ import lombok.NonNull; public class ProxyChannelInitializer extends ChannelInitializer { + protected static final String CAPTURE_HANDLER_NAME = "CaptureHandler"; + protected final IConnectionCaptureFactory connectionCaptureFactory; protected final Supplier sslEngineProvider; protected final IRootWireLoggingContext rootContext; @@ -46,7 +48,7 @@ public boolean shouldGuaranteeMessageOffloading(HttpRequest httpRequest) { } @Override - protected void initChannel(SocketChannel ch) throws IOException { + protected void initChannel(@NonNull SocketChannel ch) throws IOException { var sslContext = sslEngineProvider != null ? sslEngineProvider.get() : null; if (sslContext != null) { ch.pipeline().addLast(new SslHandler(sslEngineProvider.get())); @@ -54,7 +56,7 @@ protected void initChannel(SocketChannel ch) throws IOException { var connectionId = ch.id().asLongText(); ch.pipeline() - .addLast( + .addLast(CAPTURE_HANDLER_NAME, new ConditionallyReliableLoggingHttpHandler<>( rootContext, "", diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxySetupTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxySetupTest.java index c8ccb0d23..4e5ff226e 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxySetupTest.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/CaptureProxySetupTest.java @@ -1,6 +1,9 @@ package org.opensearch.migrations.trafficcapture.proxyserver; import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; import java.util.Properties; import org.apache.kafka.clients.CommonClientConfigs; @@ -12,6 +15,7 @@ public class CaptureProxySetupTest { public final static String kafkaBrokerString = "invalid:9092"; + public static final String TLS_PROTOCOLS_KEY = "plugins.security.ssl.http.enabled_protocols"; @Test public void testBuildKafkaPropertiesBaseCase() throws IOException { @@ -111,4 +115,41 @@ public void testBuildKafkaPropertiesWithPropertyFile() throws IOException { // Settings needed for other passed arguments (i.e. --enableMSKAuth) are ignored by property file Assertions.assertEquals("SASL_SSL", props.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } + + @Test + public void testTlsParametersAreProperlyRead() throws Exception { + for (var kvp : Map.of( + "[ TLSv1.3, TLSv1.2 ]", List.of("TLSv1.3","TLSv1.2"), + "[ TLSv1.2, TLSv1.3 ]", List.of("TLSv1.2","TLSv1.3"), + "\n - TLSv1.2\n - TLSv1.3", List.of("TLSv1.2","TLSv1.3"), + "\n - TLSv1.2", List.of("TLSv1.2")) + .entrySet()) + { + testTlsParametersAreProperlyRead(TLS_PROTOCOLS_KEY + ": " + kvp.getKey(), kvp.getValue()); + } + } + + @Test + public void testNoProtocolConfigDefaultsToSecureOnesOnly() throws Exception { + testTlsParametersAreProperlyRead("", List.of("TLSv1.2","TLSv1.3")); + } + + public void testTlsParametersAreProperlyRead(String protocolsBlockString, List expectedList) + throws Exception + { + var tempFile = Files.createTempFile("captureProxy_tlsConfig", "yaml"); + try { + Files.writeString(tempFile, "plugins.security.ssl.http.enabled: true\n" + + "plugins.security.ssl.http.pemcert_filepath: esnode.pem\n" + + "plugins.security.ssl.http.pemkey_filepath: esnode-key.pem\n" + + "plugins.security.ssl.http.pemtrustedcas_filepath: root-ca.pem\n" + + protocolsBlockString); + + var settings = CaptureProxy.getSettings(tempFile.toAbsolutePath().toString()); + Assertions.assertEquals(String.join(", ", expectedList), + String.join(", ", settings.getAsList(TLS_PROTOCOLS_KEY))); + } finally { + Files.deleteIfExists(tempFile); + } + } } diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/KafkaConfigurationCaptureProxyTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/KafkaConfigurationCaptureProxyTest.java index a7895a78e..98e85baac 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/KafkaConfigurationCaptureProxyTest.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/KafkaConfigurationCaptureProxyTest.java @@ -11,6 +11,7 @@ import org.junit.jupiter.api.Assertions; 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 org.junit.jupiter.api.function.ThrowingConsumer; import org.junit.jupiter.params.ParameterizedTest; @@ -32,6 +33,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; @Slf4j +@Tag("isolatedTest") @KafkaContainerTest @HttpdContainerTest @ToxiproxyContainerTest diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/TestHeaderRewrites.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/TestHeaderRewrites.java new file mode 100644 index 000000000..a110a3362 --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/TestHeaderRewrites.java @@ -0,0 +1,130 @@ +package org.opensearch.migrations.trafficcapture.proxyserver; + +import java.io.ByteArrayInputStream; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +import org.opensearch.migrations.testutils.HttpRequest; +import org.opensearch.migrations.testutils.SimpleHttpClientForTesting; +import org.opensearch.migrations.testutils.SimpleHttpResponse; +import org.opensearch.migrations.testutils.SimpleNettyHttpServer; +import org.opensearch.migrations.trafficcapture.proxyserver.testcontainers.CaptureProxyContainer; + +import lombok.extern.slf4j.Slf4j; + +@Slf4j +public class TestHeaderRewrites { + + public static final String ONLY_FOR_HEADERS_VALUE = "this is only for headers"; +public static final String BODY_WITH_HEADERS_CONTENTS = "\n" + + "body: should stay\n" + + "body: untouched\n" + + "body:\n"; + + @Test + public void testHeaderRewrites() throws Exception { + final var payloadBytes = "Success".getBytes(StandardCharsets.UTF_8); + final var headers = Map.of( + "Content-Type", + "text/plain", + "Content-Length", + "" + payloadBytes.length + ); + var rewriteArgs = List.of( + "--setHeader", + "host", + "localhost", + "--setHeader", + "X-new-header", + "insignificant value" + ); + var capturedRequestList = new ArrayList(); + try (var destinationServer = SimpleNettyHttpServer.makeServer(false, + Duration.ofMinutes(10), + fl -> { + capturedRequestList.add(fl); + log.trace("headers: " + fl.getHeaders().stream().map(kvp->kvp.getKey()+": "+kvp.getValue()) + .collect(Collectors.joining())); + return new SimpleHttpResponse(headers, payloadBytes, "OK", 200); + }); + var proxy = new CaptureProxyContainer(() -> destinationServer.localhostEndpoint().toString(), null, + rewriteArgs.stream()); + var client = new SimpleHttpClientForTesting()) + { + proxy.start(); + final var proxyEndpoint = CaptureProxyContainer.getUriFromContainer(proxy); + + var allHeaders = new LinkedHashMap(); + allHeaders.put("Host", "localhost"); + allHeaders.put("User-Agent", "UnitTest"); + var response = client.makeGetRequest(new URI(proxyEndpoint), allHeaders.entrySet().stream()); + var capturedRequest = capturedRequestList.get(capturedRequestList.size()-1).getHeaders().stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Assertions.assertEquals("localhost", capturedRequest.get("host")); + Assertions.assertEquals("insignificant value", capturedRequest.get("X-new-header")); + } + } + + @Test + public void testBodyDoesntRewrite() throws Exception { + final var payloadBytes = "Success".getBytes(StandardCharsets.UTF_8); + final var headers = Map.of( + "Content-Type", + "text/plain", + "Content-Length", + "" + payloadBytes.length + ); + var rewriteArgs = List.of( + "--setHeader", + "host", + "localhost", + "--setHeader", + "body", + ONLY_FOR_HEADERS_VALUE + ); + var capturedRequestList = new ArrayList(); + var capturedBodies = new ArrayList(); + try (var destinationServer = SimpleNettyHttpServer.makeNettyServer(false, + Duration.ofMinutes(10), + fullRequest -> { + var request = new SimpleNettyHttpServer.RequestToAdapter(fullRequest); + capturedRequestList.add(request); + log.atTrace().setMessage(() -> "headers: " + + request.getHeaders().stream().map(kvp->kvp.getKey()+": "+kvp.getValue()) + .collect(Collectors.joining())).log(); + capturedBodies.add(fullRequest.content().toString(StandardCharsets.UTF_8)); + return new SimpleHttpResponse(headers, payloadBytes, "OK", 200); + }); + var proxy = new CaptureProxyContainer(() -> destinationServer.localhostEndpoint().toString(), null, + rewriteArgs.stream()); + var client = new SimpleHttpClientForTesting(); + var bodyStream = new ByteArrayInputStream(BODY_WITH_HEADERS_CONTENTS.getBytes(StandardCharsets.UTF_8))) + { + proxy.start(); + final var proxyEndpoint = CaptureProxyContainer.getUriFromContainer(proxy); + + var allHeaders = new LinkedHashMap(); + allHeaders.put("Host", "localhost"); + allHeaders.put("User-Agent", "UnitTest"); + var response = client.makePutRequest(new URI(proxyEndpoint), allHeaders.entrySet().stream(), + new SimpleHttpClientForTesting.PayloadAndContentType(bodyStream, "text/plain")); + log.error("response=" + response); + var capturedRequest = capturedRequestList.get(capturedRequestList.size()-1).getHeaders().stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + Assertions.assertEquals("localhost", capturedRequest.get("host")); + Assertions.assertEquals(ONLY_FOR_HEADERS_VALUE, capturedRequest.get("body")); + + var lastBody = capturedBodies.get(capturedBodies.size()-1); + Assertions.assertEquals(BODY_WITH_HEADERS_CONTENTS, lastBody); + } + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPoolTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPoolTest.java index 89f350ee8..b4e11dae3 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPoolTest.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/ExpiringSubstitutableItemPoolTest.java @@ -13,8 +13,8 @@ import java.util.stream.IntStream; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.parallel.Isolated; import io.netty.channel.nio.NioEventLoopGroup; import io.netty.util.concurrent.DefaultPromise; @@ -24,7 +24,7 @@ import lombok.extern.slf4j.Slf4j; @Slf4j -@Isolated("Isolation based on temporal checks") +@Tag("isolatedTest") class ExpiringSubstitutableItemPoolTest { public static final int NUM_POOLED_ITEMS = 5; @@ -157,6 +157,8 @@ void get() throws Exception { Assertions.assertTrue( pool.getStats().averageWaitTime().toMillis() < pool.getStats().averageBuildTime().toMillis() ); + + eventLoop.shutdownGracefully().sync(); } private static Integer getNextItem(ExpiringSubstitutableItemPool, Integer> pool) diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandlerTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandlerTest.java new file mode 100644 index 000000000..874af2432 --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderAdderHandlerTest.java @@ -0,0 +1,92 @@ +package org.opensearch.migrations.trafficcapture.proxyserver.netty; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.IntStream; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; + +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import lombok.extern.slf4j.Slf4j; + +@WrapWithNettyLeakDetection() +@Slf4j +class HeaderAdderHandlerTest { + + private void runTestsWithSize(Supplier sizesSupplier) { + runTestWithSize("\n", sizesSupplier.get()); + runTestWithSize("\r\n", sizesSupplier.get()); + } + + @Test + public void simpleCheck() { + runTestsWithSize(() -> IntStream.of(Integer.MAX_VALUE)); + } + + @Test + public void individualBytesCheck() { + runTestsWithSize(() -> IntStream.generate(()->1)); + } + + @ParameterizedTest + @ValueSource(strings = { + "8,27,9999", + "8,12,16,999" + }) + public void fragmentedBytesCheck(String sizesStr) { + runTestsWithSize(() -> Arrays.stream(sizesStr.split(",")).mapToInt(Integer::parseInt)); + } + + private void runTestWithSize(String lineEnding, IntStream sizes) { + var extraHeader = "host: my.host"; + var newHeader = Unpooled.wrappedBuffer(extraHeader.getBytes(StandardCharsets.UTF_8)); + final var msg = makeMessage(lineEnding, ""); + + var channel = new EmbeddedChannel(new HeaderAdderHandler(newHeader)); + sliceMessageIntoChannelWrites(channel, msg, sizes); + var output = Unpooled.compositeBuffer(); + channel.inboundMessages().forEach(v -> output.addComponent(true, ((ByteBuf) v).retain())); + channel.finishAndReleaseAll(); + + Assertions.assertEquals(makeMessage(lineEnding, extraHeader + lineEnding), output.toString(StandardCharsets.UTF_8)); + output.release(); + } + + public static void sliceMessageIntoChannelWrites(EmbeddedChannel channel, String msg, IntStream sizes) { + final var lastStart = new AtomicInteger(); + sizes + .mapToObj(len -> { + var startIdx = lastStart.get(); + if (startIdx >= msg.length()) { return null; } + var endIdx = startIdx + len; + var substr = msg.substring(lastStart.get(), Math.min(endIdx, msg.length())); + lastStart.set(endIdx); + log.atTrace().setMessage(() -> "s: " + substr).log(); + return substr; + }) + .takeWhile(Objects::nonNull) + .forEach(substr -> { + var bytes = substr.getBytes(StandardCharsets.UTF_8); + var buf = channel.alloc().buffer(bytes.length); + channel.writeInbound(buf.writeBytes(bytes)); + }); + } + + String makeMessage(String lineEnding, String extraHeader) { + return "GET / HTTP/1.1" + lineEnding + + extraHeader + + "NICEHeader: v1" + lineEnding + + "silLYHeader: yyy" + lineEnding + + lineEnding; + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandlerTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandlerTest.java new file mode 100644 index 000000000..133057c06 --- /dev/null +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/HeaderRemoverHandlerTest.java @@ -0,0 +1,164 @@ +package org.opensearch.migrations.trafficcapture.proxyserver.netty; + +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.Random; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import org.opensearch.migrations.testutils.WrapWithNettyLeakDetection; + +import io.netty.buffer.ByteBuf; +import io.netty.channel.embedded.EmbeddedChannel; +import lombok.extern.slf4j.Slf4j; + +@WrapWithNettyLeakDetection(repetitions = 1) +@Slf4j +class HeaderRemoverHandlerTest { + + private static final int NUM_RANDOM_RUNS = 1_000; + + private void runTestsWithSize(BiFunction msgMaker, Supplier sizesSupplier) { + log.atDebug().setMessage(() -> + "sizes: " + sizesSupplier.get().limit(16).mapToObj(i->""+i) + .collect(Collectors.joining(","))).log(); + runTestWithSize(b -> msgMaker.apply(b,"\n"), sizesSupplier.get()); + runTestWithSize(b -> msgMaker.apply(b, "\r\n"), sizesSupplier.get()); + } + + public void runTestWithSize(Function messageMaker, IntStream sizes) { + final var sourceMsg = messageMaker.apply(true); + + var channel = new EmbeddedChannel(new HeaderRemoverHandler("host:")); + HeaderAdderHandlerTest.sliceMessageIntoChannelWrites(channel, sourceMsg, sizes); + var outputBuf = channel.alloc().compositeBuffer(); + channel.inboundMessages().forEach(v -> outputBuf.addComponent(true, ((ByteBuf) v).retain())); + channel.finishAndReleaseAll(); + + var outputString = outputBuf.toString(StandardCharsets.UTF_8); + Assertions.assertEquals(messageMaker.apply(false), outputString, + "Error converting source message: " + sourceMsg); + outputBuf.release(); + } + + @Test + public void newlinesArePreserved() { + runTestsWithSize((b,s) -> "GET / HTTP/1.1\r\n" + (b ? "host: localhost\r\n" : "") + "\r\n", + () -> IntStream.of(Integer.MAX_VALUE)); + } + + @Test + public void throwsOnHostFormatError() { + Assertions.assertThrows(IllegalArgumentException.class, () -> new HeaderRemoverHandler("host")); + Assertions.assertThrows(IllegalArgumentException.class, () -> new HeaderRemoverHandler("h: ")); + } + + @Test + public void simpleCheck() { + runTestsWithSize(HeaderRemoverHandlerTest::makeInterlacedMessage, () -> IntStream.of(Integer.MAX_VALUE)); + } + + @Test + public void trivialSingleByte() { + runTestsWithSize((x,y) -> "A", () -> IntStream.generate(() -> 1)); + } + + @Test + public void individualBytesCheck() { + runTestsWithSize(HeaderRemoverHandlerTest::makeThinMessage, () -> IntStream.generate(() -> 1)); + runTestsWithSize(HeaderRemoverHandlerTest::makeInterlacedMessage, () -> IntStream.generate(() -> 1)); + runTestsWithSize(HeaderRemoverHandlerTest::makeConsecutiveMessage, () -> IntStream.generate(() -> 1)); + } + + @ParameterizedTest + @ValueSource(strings = { + "8,22,22,22,22,9999" + }) + public void fragmentedCheckInterlaced(String sizesStr) { + runTestsWithSize(HeaderRemoverHandlerTest::makeInterlacedMessage, + () -> Arrays.stream(sizesStr.split(",")).mapToInt(Integer::parseInt)); + } + + @Test + @WrapWithNettyLeakDetection(repetitions = 1) + public void randomFragmentedCheckInterlaced() { + final var bound = getBound(HeaderRemoverHandlerTest::makeInterlacedMessage); + for (int i=0; i "random run={}").addArgument(i).log(); + runTestsWithSize(HeaderRemoverHandlerTest::makeInterlacedMessage, + () -> IntStream.generate(() -> r.nextInt(bound))); + } + } + + @ParameterizedTest + @ValueSource(strings = { + "8,22,22,22,22,9999" + }) + public void fragmentedCheckConsecutive(String sizesStr) { + runTestsWithSize(HeaderRemoverHandlerTest::makeConsecutiveMessage, + () -> Arrays.stream(sizesStr.split(",")).mapToInt(Integer::parseInt)); + } + + @Test + @WrapWithNettyLeakDetection(repetitions = 1) + public void randomFragmentedCheckConsecutive() { + final var bound = getBound(HeaderRemoverHandlerTest::makeConsecutiveMessage); + for (int i=0; i "random run={}").addArgument(i).log(); + runTestsWithSize(HeaderRemoverHandlerTest::makeConsecutiveMessage, + () -> IntStream.generate(() -> r.nextInt(bound))); + } + } + + private int getBound(BiFunction makeInterlacedMessage) { + return Arrays.stream(makeInterlacedMessage.apply(true, "\n").split("\n")) + .mapToInt(String::length) + .map(x->x*2) + .max() + .orElseThrow(() -> new IllegalStateException("No lines in the sample")); + } + + static String makeInterlacedMessage(boolean withHosts, String lineEnding) { + return "GET / HTTP/1.1" + lineEnding + + "hoststays: v1" + lineEnding + + (withHosts ? ("HOST: begone" + lineEnding) : "") + + "different: v2" + lineEnding + + (withHosts ? ("HosT: begone" + lineEnding) : "") + + "keeper: v3" + lineEnding + + lineEnding; + } + + static String makeThinMessage(boolean withHosts, String lineEnding) { + return "G" + lineEnding + + "h: a" + lineEnding + + (withHosts ? ("HOST: b" + lineEnding) : "") + + "d: c" + lineEnding + + (withHosts ? ("HosT: e" + lineEnding) : "") + + lineEnding; + } + + + static String makeConsecutiveMessage(boolean withHosts, String lineEnding) { + return "GET / HTTP/1.1" + lineEnding + + "hoststays: a1" + lineEnding + + "different: b2" + lineEnding + + (withHosts ? ("HOST: strike" + lineEnding) : "") + + (withHosts ? ("HosT: begone" + lineEnding) : "") + + "e2: c3" + lineEnding + + "hos: d4" + lineEnding + + (withHosts ? ("HOST: foo" + lineEnding) : "") + + (withHosts ? ("HosT: bar" + lineEnding) : "") + + "X: Y" + lineEnding + + lineEnding; + } +} diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java index 6cfd4ec01..7523583c2 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/netty/NettyScanningHttpProxyTest.java @@ -18,7 +18,7 @@ import org.junit.jupiter.api.Test; import org.opensearch.common.collect.Tuple; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.PortFinder; import org.opensearch.migrations.testutils.SimpleHttpClientForTesting; import org.opensearch.migrations.testutils.SimpleHttpResponse; @@ -207,7 +207,8 @@ private static Tuple startServers( var connectionPool = new BacksideConnectionPool(testServerUri, null, 10, Duration.ofSeconds(10)); nshp.get() - .start(rootCtx, connectionPool, 1, null, connectionCaptureFactory, new RequestCapturePredicate()); + .start(new ProxyChannelInitializer(rootCtx, connectionPool, null, + connectionCaptureFactory, new RequestCapturePredicate()), 1); System.out.println("proxy port = " + port); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -217,7 +218,7 @@ private static Tuple startServers( return new Tuple<>(nshp.get(), underlyingPort); } - private static SimpleHttpResponse makeContext(HttpRequestFirstLine request) { + private static SimpleHttpResponse makeContext(HttpRequest request) { var headers = Map.of( "Content-Type", "text/plain", diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/CaptureProxyContainer.java b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/CaptureProxyContainer.java index 6bafcd3d0..7d03179d3 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/CaptureProxyContainer.java +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/java/org/opensearch/migrations/trafficcapture/proxyserver/testcontainers/CaptureProxyContainer.java @@ -6,6 +6,7 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.opensearch.migrations.testutils.PortFinder; import org.opensearch.migrations.trafficcapture.proxyserver.CaptureProxy; @@ -26,20 +27,24 @@ public class CaptureProxyContainer extends GenericContainer implements AutoClose private static final Duration TIMEOUT_DURATION = Duration.ofSeconds(30); private final Supplier destinationUriSupplier; private final Supplier kafkaUriSupplier; + private final List extraArgs; private Integer listeningPort; private Thread serverThread; - public CaptureProxyContainer( - final Supplier destinationUriSupplier, - final Supplier kafkaUriSupplier - ) { + public CaptureProxyContainer(final Supplier destinationUriSupplier, + final Supplier kafkaUriSupplier, + Stream extraArgs) { this.destinationUriSupplier = destinationUriSupplier; this.kafkaUriSupplier = kafkaUriSupplier; + this.extraArgs = extraArgs.collect(Collectors.toList()); + } + + public CaptureProxyContainer(Supplier destinationUriSupplier, Supplier kafkaUriSupplier) { + this(destinationUriSupplier, kafkaUriSupplier, Stream.of()); } public CaptureProxyContainer(final String destinationUri, final String kafkaUri) { - this.destinationUriSupplier = () -> destinationUri; - this.kafkaUriSupplier = () -> kafkaUri; + this(() -> destinationUri, () -> kafkaUri); } public CaptureProxyContainer(final Container destination, final KafkaContainer kafka) { @@ -74,6 +79,8 @@ public void start() { argsList.add(String.valueOf(listeningPort)); argsList.add("--insecureDestination"); + argsList.addAll(extraArgs); + CaptureProxy.main(argsList.toArray(new String[0])); } catch (Exception e) { throw new AssertionError("Should not have exception", e); diff --git a/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties b/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties index 702836711..da3da9bb4 100644 --- a/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties +++ b/TrafficCapture/trafficCaptureProxyServer/src/test/resources/log4j2.properties @@ -1,4 +1,6 @@ -status = error +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} appender.console.type = Console appender.console.name = STDERR @@ -6,6 +8,12 @@ appender.console.target = SYSTEM_ERR appender.console.layout.type = PatternLayout appender.console.layout.pattern = [%-5level] %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} [%t] %c{1} - %msg%equals{ ctx=%mdc}{ ctx=\{\}}{}%n -rootLogger.level = debug +rootLogger.level = info rootLogger.appenderRefs = stderr rootLogger.appenderRef.stderr.ref = STDERR + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/trafficCaptureProxyServerTest/build.gradle b/TrafficCapture/trafficCaptureProxyServerTest/build.gradle index 6ca1da941..410859562 100644 --- a/TrafficCapture/trafficCaptureProxyServerTest/build.gradle +++ b/TrafficCapture/trafficCaptureProxyServerTest/build.gradle @@ -31,7 +31,7 @@ dependencies { implementation group: 'org.apache.jmeter', name: 'ApacheJMeter_core', withoutBom implementation group: 'org.apache.jmeter', name: 'ApacheJMeter_http', withoutBom implementation group: 'org.apache.jmeter', name: 'ApacheJMeter_config', withoutBom - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' } CommonConfigurations.applyCommonConfigurations(project) diff --git a/TrafficCapture/trafficReplayer/README.md b/TrafficCapture/trafficReplayer/README.md index 0b74e2425..06cff0b36 100644 --- a/TrafficCapture/trafficReplayer/README.md +++ b/TrafficCapture/trafficReplayer/README.md @@ -63,7 +63,7 @@ which has comments throughout it to indicate how data percolates and is converte ## Handlers -With the exception of the preparation around JSON model and its transformation, all the other handlers (compression, +Except for the conversions around JSON payloads, all the other handlers (compression, chunked, and JSON parsing/serialization), use streaming data models via mostly custom handlers. This should minimize the memory load (working set size, cache misses, etc). However, attempts have not yet been made to reduce the number of allocations. Those optimization may not have extremely high value, especially when JSON parsing will need to create diff --git a/TrafficCapture/trafficReplayer/build.gradle b/TrafficCapture/trafficReplayer/build.gradle index 6ecf7a9e5..f15c13984 100644 --- a/TrafficCapture/trafficReplayer/build.gradle +++ b/TrafficCapture/trafficReplayer/build.gradle @@ -27,7 +27,7 @@ dependencies { runtimeOnly project(':TrafficCapture:transformationPlugins:jsonMessageTransformers:jsonJoltMessageTransformerProvider') runtimeOnly project(':TrafficCapture:transformationPlugins:jsonMessageTransformers:openSearch23PlusTargetTransformerProvider') - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'com.fasterxml.jackson.core', name: 'jackson-databind' implementation group: 'com.google.guava', name: 'guava' implementation group: 'com.google.protobuf', name: 'protobuf-java' diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java index ec3a94c8f..92be443c7 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest.java @@ -10,6 +10,7 @@ import java.util.stream.IntStream; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -23,6 +24,7 @@ import lombok.extern.slf4j.Slf4j; @Slf4j +@Tag("longTest") public class ExhaustiveCapturedTrafficToHttpTransactionAccumulatorTest extends InstrumentationTest { public static Arguments[] generateAllTestsAndConfirmComplete(IntStream seedStream) { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java index 930118c23..610c63baa 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/ResultsToLogsConsumerTest.java @@ -15,6 +15,7 @@ import org.apache.logging.log4j.core.impl.Log4jContextFactory; import org.apache.logging.log4j.core.selector.ClassLoaderContextSelector; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.parallel.ResourceLock; @@ -116,6 +117,7 @@ private static byte[] loadResourceAsBytes(String path) throws IOException { } @Test + @Tag("longTest") @ResourceLock("TestContext") public void testOutputterForGet() throws IOException { final String EXPECTED_LOGGED_OUTPUT = "" @@ -174,6 +176,7 @@ public void testOutputterForGet() throws IOException { } @Test + @Tag("longTest") @ResourceLock("TestContext") public void testOutputterForPost() throws IOException { final String EXPECTED_LOGGED_OUTPUT = "" diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java index 5fb501c5f..dd23317e9 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/TrafficReplayerTest.java @@ -183,6 +183,7 @@ static byte[] synthesizeTrafficStreamsIntoByteArray(Instant timestamp, int numSt } @Test + @WrapWithNettyLeakDetection(repetitions = 1) public void testReader() throws Exception { var uri = new URI("http://localhost:9200"); try ( @@ -256,6 +257,7 @@ public void onTrafficStreamIgnored( @Test @Tag("longTest") + @WrapWithNettyLeakDetection(repetitions = 2) public void testCapturedReadsAfterCloseAreHandledAsNew() throws Exception { var uri = new URI("http://localhost:9200"); try ( diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java index cc9683e1c..37d157f15 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/JsonAccumulatorTest.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -69,13 +70,24 @@ byte[] getData(String key) throws IOException { @ParameterizedTest @CsvSource({ "tiny,2", - "tiny,20000", + "tiny,20000"}) + public void testAccumulationShort(String dataName, int chunkBound) throws IOException { + testAccumulation(dataName, chunkBound); + } + + @ParameterizedTest + @Tag("longTest") + @CsvSource({ "medium,2", "medium,20000", "large,2", "large,20000", "largeAndPacked,2", "largeAndPacked,20000" }) + public void testAccumulationLong(String dataName, int chunkBound) throws IOException { + testAccumulation(dataName, chunkBound); + } + public void testAccumulation(String dataName, int chunkBound) throws IOException { var testFileBytes = getData(dataName); var outputJson = readJson(testFileBytes, 2); @@ -84,6 +96,5 @@ public void testAccumulation(String dataName, int chunkBound) throws IOException var jacksonParsedRoundTripped = mapper.writeValueAsString(mapper.readTree(testFileBytes)); var jsonAccumParsedRoundTripped = mapper.writeValueAsString(outputJson); Assertions.assertEquals(jacksonParsedRoundTripped, jsonAccumParsedRoundTripped); - } } diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java index a787f52b7..6f1900268 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/NettyPacketToHttpConsumerTest.java @@ -35,7 +35,7 @@ import org.opensearch.migrations.replay.http.retries.NoRetryEvaluatorFactory; import org.opensearch.migrations.replay.traffic.source.BufferedFlowController; import org.opensearch.migrations.replay.util.TextTrackedFuture; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.SimpleHttpClientForTesting; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; @@ -85,7 +85,7 @@ protected TestContext makeInstrumentationContext() { return TestContext.withTracking(false, true); } - private static SimpleHttpResponse makeResponseContext(HttpRequestFirstLine request) { + private static SimpleHttpResponse makeResponseContext(HttpRequest request) { var headers = new TreeMap( Map.of( "Content-Type", @@ -100,7 +100,7 @@ private static SimpleHttpResponse makeResponseContext(HttpRequestFirstLine reque return new SimpleHttpResponse(headers, payloadBytes, "OK", 200); } - private static SimpleHttpResponse makeResponseContextLarge(HttpRequestFirstLine request) { + private static SimpleHttpResponse makeResponseContextLarge(HttpRequest request) { var headers = new TreeMap( Map.of( "Content-Type", @@ -195,7 +195,6 @@ public void testHttpResponseIsSuccessfullyCaptured(boolean useTls, boolean large @CsvSource({ "false, false", "false, true", "true, false", "true, true" }) @Tag("longTest") @WrapWithNettyLeakDetection(repetitions = 1) - @Tag("longTest") public void testThatPeerResetTriggersFinalizeFuture(boolean useTls, boolean withServerReadTimeout) throws Exception { final var RESPONSE_TIMEOUT_FOR_HUNG_TEST = Duration.ofMillis(500); @@ -285,6 +284,7 @@ private void testPeerResets( @ParameterizedTest @CsvSource({ "false, false", "false, true", "true, false", "true, true" }) + @WrapWithNettyLeakDetection(repetitions = 1) @Tag("longTest") public void testThatConnectionsAreKeptAliveAndShared(boolean useTls, boolean largeResponse) throws Exception { try ( @@ -396,6 +396,7 @@ public void testMetricCountsFor_testThatConnectionsAreKeptAliveAndShared(boolean } @ParameterizedTest + @Tag("longTest") @CsvSource({ "false", "true" }) public void testResponseTakesLongerThanTimeout(boolean useTls) throws Exception { var responseTimeout = Duration.ofMillis(50); @@ -439,7 +440,7 @@ public void testResponseTakesLongerThanTimeout(boolean useTls) throws Exception Instant.now(), Instant.now(), () -> Stream.of(EXPECTED_REQUEST_STRING.getBytes(StandardCharsets.UTF_8))); - var maxTimeToWaitForTimeoutOrResponse = Duration.ofSeconds(10); + var maxTimeToWaitForTimeoutOrResponse = Duration.ofSeconds(30); var aggregatedResponse = requestFinishFuture.get(maxTimeToWaitForTimeoutOrResponse); log.atInfo().setMessage("RequestFinishFuture finished").log(); Assertions.assertInstanceOf(ReadTimeoutException.class, aggregatedResponse.getError()); @@ -447,6 +448,7 @@ public void testResponseTakesLongerThanTimeout(boolean useTls) throws Exception } @ParameterizedTest + @Tag("longTest") @CsvSource({ "false", "true" }) public void testTimeBetweenRequestsLongerThanResponseTimeout(boolean useTls) throws Exception { var responseTimeout = Duration.ofMillis(100); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java index 838e73cb1..3774f64e7 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/datahandlers/http/HttpJsonTransformingConsumerTest.java @@ -11,6 +11,7 @@ import java.util.stream.Stream; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -48,8 +49,17 @@ private static Stream provideTestParameters() { ); } + @Test + @WrapWithNettyLeakDetection(repetitions = 2) + public void testSomeRequestProcessing() throws Exception { + var args = provideTestParameters().findFirst().get(); + testRequestProcessing((Integer) args.get()[0], (Boolean) args.get()[1], (String) args.get()[2]); + } + @ParameterizedTest @MethodSource("provideTestParameters") + @Tag("longTest") + @WrapWithNettyLeakDetection(repetitions = 2) public void testRequestProcessing(Integer attemptedChunks, Boolean hostTransformation, String requestFile) throws Exception { final var dummyAggregatedResponse = new AggregatedRawResponse(null, 17, Duration.ZERO, List.of(), null); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java index 3be4b095c..4d7f7d760 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullReplayerWithTracingChecksTest.java @@ -52,6 +52,7 @@ protected TestContext makeInstrumentationContext() { @ParameterizedTest @ValueSource(ints = { 1, 2 }) @ResourceLock("TrafficReplayerRunner") + // run in isolation to reduce the chance that there's a broken connection, upsetting the tcpConnection count check @Tag("longTest") public void testStreamWithRequestsWithCloseIsCommittedOnce(int numRequests) throws Throwable { var random = new Random(1); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java index e159ddee6..eb31a7762 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/FullTrafficReplayerTest.java @@ -316,8 +316,8 @@ public CommitResult commitTrafficStream(ITrafficStreamKey trafficStreamKey) thro } @Test + @Tag("longTest") public void makeSureThatCollateralDamageDoesntFreezeTests() throws Throwable { - var imposterThread = new Thread(() -> { while (true) { try { diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java index c240858e4..c5ecf5d03 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/e2etests/SlowAndExpiredTrafficStreamBecomesTwoTargetChannelsTest.java @@ -27,7 +27,7 @@ import org.opensearch.migrations.replay.traffic.source.ArrayCursorTrafficCaptureSource; import org.opensearch.migrations.replay.traffic.source.ArrayCursorTrafficSourceContext; import org.opensearch.migrations.replay.traffic.source.BlockingTrafficSource; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.SimpleHttpResponse; import org.opensearch.migrations.testutils.SimpleNettyHttpServer; import org.opensearch.migrations.tracing.InMemoryInstrumentationBundle; @@ -172,7 +172,7 @@ static String makePath(String connection, int i) { return "/" + connection + "/" + Integer.toString(i); } - private static class TrackingResponseBuilder implements Function { + private static class TrackingResponseBuilder implements Function { List pathsReceivedList; CountDownLatch targetRequestsPending; @@ -182,8 +182,8 @@ public TrackingResponseBuilder(int expected) { } @Override - public SimpleHttpResponse apply(HttpRequestFirstLine firstLine) { - var pathReceived = firstLine.path().getPath(); + public SimpleHttpResponse apply(HttpRequest firstLine) { + var pathReceived = firstLine.getPath().getPath(); pathsReceivedList.add(pathReceived); var payloadBytes = pathReceived.getBytes(StandardCharsets.UTF_8); targetRequestsPending.countDown(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java index c601597bb..feff8bc01 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/http/retries/HttpRetryTest.java @@ -149,8 +149,8 @@ public void testPersistentRequestFailuresAreRetriedThenFailed() throws Exception } } - @Tag("longTest") @Test + @Tag("longTest") @WrapWithNettyLeakDetection(disableLeakChecks = true) // code is forcibly terminated so leaks are expected public void testConnectionFailuresNeverGiveUp() throws Exception { URI serverUri; @@ -218,7 +218,7 @@ public void testMalformedResponseFailuresNeverGiveUp() throws Exception { var server = new GenericContainer<>(HTTPD_IMAGE) .withNetwork(network) .withNetworkAliases(SERVERNAME_ALIAS) - .waitingFor(Wait.forHttp("/").forStatusCode(200)); + .waitingFor(Wait.forHttp("/").forStatusCode(200)).withStartupTimeout(Duration.ofMinutes(5)); var toxiproxy = new ToxiProxyWrapper(network)) { server.start(); diff --git a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java index 2a824cb33..8aef4ed69 100644 --- a/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java +++ b/TrafficCapture/trafficReplayer/src/test/java/org/opensearch/migrations/replay/kafka/KafkaTrafficCaptureSourceLongTermTest.java @@ -36,7 +36,7 @@ public class KafkaTrafficCaptureSourceLongTermTest extends InstrumentationTest { ); @Test - @Tag("longTest") + @Tag("isolatedTest") public void testTrafficCaptureSource() throws Exception { String testTopicName = "TEST_TOPIC"; diff --git a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties index c97fd4ccb..7136cf5aa 100644 --- a/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties +++ b/TrafficCapture/trafficReplayer/src/test/resources/log4j2.properties @@ -1,4 +1,6 @@ -status = error +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} # Root logger options rootLogger.level = info @@ -14,3 +16,9 @@ appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t # of the logs for tests logger.OutputTupleJsonLogger.name = OutputTupleJsonLogger logger.OutputTupleJsonLogger.level = OFF + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java index 2311de00c..fd2e32857 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestCapturePacketToHttpHandler.java @@ -45,19 +45,20 @@ public TestCapturePacketToHttpHandler( @Override public TrackedFuture consumeBytes(ByteBuf nextRequestPacket) { numConsumes.incrementAndGet(); - log.info("incoming buffer refcnt=" + nextRequestPacket.refCnt()); + log.atDebug().setMessage(()->"incoming buffer refcnt=" + nextRequestPacket.refCnt()).log(); var duplicatedPacket = nextRequestPacket.retainedDuplicate(); return new TrackedFuture<>(CompletableFuture.runAsync(() -> { try { - log.info("Running async future for " + nextRequestPacket); + log.atDebug().setMessage(()->"Running async future for " + nextRequestPacket).log(); Thread.sleep(consumeDuration.toMillis()); - log.info("woke up from sleeping for " + nextRequestPacket); + log.atDebug().setMessage(()->"woke up from sleeping for " + nextRequestPacket).log(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw Lombok.sneakyThrow(e); } try { - log.info("At the time of committing the buffer, refcnt=" + duplicatedPacket.refCnt()); + log.atDebug() + .setMessage(()->"At the time of committing the buffer, refcnt=" + duplicatedPacket.refCnt()).log(); duplicatedPacket.readBytes(byteArrayOutputStream, nextRequestPacket.readableBytes()); duplicatedPacket.release(); } catch (IOException e) { diff --git a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestHttpServerContext.java b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestHttpServerContext.java index 055463fea..00bdf7205 100644 --- a/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestHttpServerContext.java +++ b/TrafficCapture/trafficReplayer/src/testFixtures/java/org/opensearch/migrations/replay/TestHttpServerContext.java @@ -5,7 +5,7 @@ import java.util.Map; import java.util.Random; -import org.opensearch.migrations.testutils.HttpRequestFirstLine; +import org.opensearch.migrations.testutils.HttpRequest; import org.opensearch.migrations.testutils.SimpleHttpResponse; import lombok.Lombok; @@ -30,18 +30,18 @@ public static String getRequestStringForSimpleGet(String uriPath) { ); } - public static SimpleHttpResponse makeResponse(Random rand, HttpRequestFirstLine firstLine) { + public static SimpleHttpResponse makeResponse(Random rand, HttpRequest firstLine) { return makeResponse(firstLine, Duration.ofMillis(rand.nextInt(MAX_RESPONSE_TIME_MS))); } - public static SimpleHttpResponse makeResponse(HttpRequestFirstLine r, Duration responseWaitTime) { + public static SimpleHttpResponse makeResponse(HttpRequest r, Duration responseWaitTime) { try { Thread.sleep(responseWaitTime.toMillis()); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw Lombok.sneakyThrow(e); } - String body = SERVER_RESPONSE_BODY_PREFIX + r.path(); + String body = SERVER_RESPONSE_BODY_PREFIX + r.getPath(); var payloadBytes = body.getBytes(StandardCharsets.UTF_8); var headers = Map.of( "Content-Type", diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJMESPathMessageTransformerProvider/src/test/resources/log4j2.properties b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJMESPathMessageTransformerProvider/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJMESPathMessageTransformerProvider/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/resources/log4j2.properties b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/jsonJoltMessageTransformerProvider/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/resources/log4j2.properties b/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/TrafficCapture/transformationPlugins/jsonMessageTransformers/openSearch23PlusTargetTransformerProvider/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/awsUtilities/src/test/resources/log4j2.properties b/awsUtilities/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/awsUtilities/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/build.gradle b/build.gradle index dd989456e..1e0efd760 100644 --- a/build.gradle +++ b/build.gradle @@ -70,6 +70,7 @@ subprojects { showCauses true showStackTraces true } + maxParallelForks = gradle.startParameter.maxWorkerCount // Provide way to exclude particular tests from CLI // e.g. ../gradlew test -PexcludeTests=**/KafkaProtobufConsumerLongTermTest* @@ -83,21 +84,36 @@ subprojects { systemProperty 'log4j2.contextSelector', 'org.apache.logging.log4j.core.selector.BasicContextSelector' // Verify assertions in tests jvmArgs '-ea' + jacoco.enabled = true } // Mutually exclusive tests to avoid duplication tasks.named('test') { + systemProperty 'migrationLogLevel', 'TRACE' useJUnitPlatform { - excludeTags 'longTest' + excludeTags('longTest', 'isolatedTest') } - jacoco.enabled = true } tasks.register('slowTest', Test) { + systemProperty 'migrationLogLevel', 'DEBUG' useJUnitPlatform { includeTags 'longTest' + excludeTags 'isolatedTest' + } + } + + tasks.register('isolatedTest', Test) { + maxParallelForks = 1 + useJUnitPlatform { + includeTags 'isolatedTest' } - jacoco.enabled = true + } + + tasks.register('fullTest') { + dependsOn test + dependsOn slowTest + dependsOn isolatedTest } task javadocJar(type: Jar, dependsOn: javadoc) { @@ -199,6 +215,36 @@ subprojects { } } +gradle.projectsEvaluated { + List isolatedTestsTasks = [] + List sharedProcessTestsTasks = [] + subprojects { subproject -> + subproject.tasks.withType(Test).all { task -> + if (task.name == "isolatedTest") { + isolatedTestsTasks.add(task) + } else { + sharedProcessTestsTasks.add(task) + } + } + } + isolatedTestsTasks.sort { task -> task.project.name } + + // Create a sequential dependency chain + Task previousTask = null + isolatedTestsTasks.each { task -> + sharedProcessTestsTasks.forEach {task.mustRunAfter(it) } + if (previousTask != null) { + task.mustRunAfter(previousTask) + } + previousTask = task + } + + tasks.register("allTests") { + dependsOn sharedProcessTestsTasks + dependsOn isolatedTestsTasks + } +} + task mergeJacocoReports(type: JacocoReport) { def jacocoReportTasks = subprojects.collect { it.tasks.withType(JacocoReport).matching { it.name == "jacocoTestReport" } }.flatten() dependsOn jacocoReportTasks diff --git a/commonDependencyVersionConstraints/build.gradle b/commonDependencyVersionConstraints/build.gradle index 4d93ce825..9dadff64d 100644 --- a/commonDependencyVersionConstraints/build.gradle +++ b/commonDependencyVersionConstraints/build.gradle @@ -45,7 +45,7 @@ dependencies { api group: 'org.apache.jmeter', name: 'ApacheJMeter_http', version: jmeter api group: 'org.apache.jmeter', name: 'ApacheJMeter_config', version: jmeter - api group: 'com.beust', name: 'jcommander', version: '1.82' + api group: 'org.jcommander', name: 'jcommander', version: '1.85' api group: 'org.projectlombok', name: 'lombok', version: '1.18.28' @@ -104,9 +104,9 @@ dependencies { api group: 'eu.rekawek.toxiproxy', name: 'toxiproxy-java', version: '2.1.7' - def lucene = '8.11.3'; + def lucene = '9.11.1'; api group: 'org.apache.lucene', name: 'lucene-core', version: lucene - api group: 'org.apache.lucene', name: 'lucene-analyzers-common', version: lucene + api group: 'org.apache.lucene', name: 'lucene-analysis-common', version: lucene api group: 'org.apache.lucene', name: 'lucene-backward-codecs', version: lucene api group: 'org.hamcrest', name: 'hamcrest', version: '2.2' diff --git a/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java b/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java index 0d7d45b69..504e534a0 100644 --- a/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java +++ b/coreUtilities/src/main/java/org/opensearch/migrations/tracing/CommonScopedMetricInstruments.java @@ -70,7 +70,7 @@ public CommonScopedMetricInstruments(Meter meter, ScopeLabels stockMetricLabels, } private static List getBuckets(double firstBucketSize, double lastBucketCeiling) { - var buckets = getExponentialBucketsBetween(firstBucketSize, lastBucketCeiling, 2.0); + var buckets = getExponentialBucketsBetween(firstBucketSize, lastBucketCeiling); log.atTrace() .setMessage( () -> "Setting buckets to " diff --git a/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java b/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java index 3e869cef3..3aafc7876 100644 --- a/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java +++ b/coreUtilities/src/main/java/org/opensearch/migrations/tracing/RootOtelContext.java @@ -177,7 +177,6 @@ private static Span buildSpanWithParent(SpanBuilder builder, Span parentSpan, St String spanName, Stream linkedSpans ) { - assert forScope.getCurrentSpan() == null; var forEnclosingScope = forScope.getEnclosingScope(); var parentSpan = forEnclosingScope == null ? null : forEnclosingScope.getCurrentSpan(); var spanBuilder = getOpenTelemetry().getTracer(scopeName).spanBuilder(spanName); diff --git a/coreUtilities/src/main/java/org/opensearch/migrations/utils/ProcessHelpers.java b/coreUtilities/src/main/java/org/opensearch/migrations/utils/ProcessHelpers.java index d2d2dce02..6e4957755 100644 --- a/coreUtilities/src/main/java/org/opensearch/migrations/utils/ProcessHelpers.java +++ b/coreUtilities/src/main/java/org/opensearch/migrations/utils/ProcessHelpers.java @@ -7,7 +7,9 @@ @Slf4j public class ProcessHelpers { - private static final String DEFAULT_NODE_ID = "generated_" + UUID.randomUUID().toString(); + private static final String DEFAULT_NODE_ID = "generated_" + UUID.randomUUID(); + + private ProcessHelpers() {} public static String getNodeInstanceName() { var nodeId = Optional.of("ECS_TASK_ID").map(System::getenv) diff --git a/coreUtilities/src/test/resources/log4j2.properties b/coreUtilities/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/coreUtilities/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/dashboardsSanitizer/build.gradle b/dashboardsSanitizer/build.gradle index 0b8c8a8c4..894c7e051 100644 --- a/dashboardsSanitizer/build.gradle +++ b/dashboardsSanitizer/build.gradle @@ -29,7 +29,7 @@ dependencies { implementation group: 'org.slf4j', name: 'slf4j-api' implementation group: 'org.apache.logging.log4j', name: 'log4j-slf4j2-impl' - implementation group: 'com.beust', name: 'jcommander' + implementation group: 'org.jcommander', name: 'jcommander' implementation group: 'org.semver4j', name: 'semver4j' testImplementation group: 'org.mockito', name: 'mockito-core' diff --git a/dashboardsSanitizer/src/test/resources/log4j2.properties b/dashboardsSanitizer/src/test/resources/log4j2.properties new file mode 100644 index 000000000..4f87b2f62 --- /dev/null +++ b/dashboardsSanitizer/src/test/resources/log4j2.properties @@ -0,0 +1,18 @@ +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-INFO} + +appender.console.type = Console +appender.console.name = Console +appender.console.target = SYSTEM_OUT +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%n + +rootLogger.level = info +rootLogger.appenderRef.console.ref = Console + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel} diff --git a/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts b/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts index 619b3acc4..2ce5358bd 100644 --- a/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts +++ b/deployment/cdk/opensearch-service-migration/lib/common-utilities.ts @@ -326,6 +326,13 @@ export class ClusterSigV4Auth { this.region = region; this.serviceSigningName = service; } + + toDict() { + return { + region: this.region, + service: this.serviceSigningName + } + } } export class ClusterBasicAuth { @@ -379,7 +386,7 @@ export class ClusterAuth { return {no_auth: ""}; } if (this.sigv4) { - return {sigv4: this.sigv4}; + return {sigv4: this.sigv4.toDict()}; } return {}; } diff --git a/deployment/cdk/opensearch-service-migration/lib/migration-assistance-stack.ts b/deployment/cdk/opensearch-service-migration/lib/migration-assistance-stack.ts index c47524fd4..a9a0042df 100644 --- a/deployment/cdk/opensearch-service-migration/lib/migration-assistance-stack.ts +++ b/deployment/cdk/opensearch-service-migration/lib/migration-assistance-stack.ts @@ -1,6 +1,6 @@ import {RemovalPolicy, Stack} from "aws-cdk-lib"; import {IVpc, Port, SecurityGroup, SubnetFilter, SubnetType} from "aws-cdk-lib/aws-ec2"; -import {FileSystem} from 'aws-cdk-lib/aws-efs'; +import {FileSystem, LifecyclePolicy, ThroughputMode} from 'aws-cdk-lib/aws-efs'; import {Construct} from "constructs"; import {CfnConfiguration} from "aws-cdk-lib/aws-msk"; import {Cluster} from "aws-cdk-lib/aws-ecs"; @@ -8,7 +8,11 @@ import {StackPropsExt} from "./stack-composer"; import {LogGroup, RetentionDays} from "aws-cdk-lib/aws-logs"; import {StreamingSourceType} from "./streaming-source-type"; import {Bucket, BucketEncryption} from "aws-cdk-lib/aws-s3"; -import {createMigrationStringParameter, MigrationSSMParameter, parseRemovalPolicy} from "./common-utilities"; +import { + createMigrationStringParameter, + MigrationSSMParameter, + parseRemovalPolicy +} from "./common-utilities"; import { ClientAuthentication, ClientBrokerEncryption, @@ -188,7 +192,9 @@ export class MigrationAssistanceStack extends Stack { const sharedLogsEFS = new FileSystem(this, 'sharedLogsEFS', { vpc: props.vpc, securityGroup: sharedLogsSG, - removalPolicy: replayerEFSRemovalPolicy + removalPolicy: replayerEFSRemovalPolicy, + lifecyclePolicy: LifecyclePolicy.AFTER_1_DAY, // Cost break even is at 26 downloads / month + throughputMode: ThroughputMode.BURSTING, // Best cost characteristics for write heavy, short-lived data }); createMigrationStringParameter(this, sharedLogsEFS.fileSystemId, { ...props, diff --git a/deployment/cdk/opensearch-service-migration/lib/migration-services-yaml.ts b/deployment/cdk/opensearch-service-migration/lib/migration-services-yaml.ts index be9bf37ce..68fb25cc3 100644 --- a/deployment/cdk/opensearch-service-migration/lib/migration-services-yaml.ts +++ b/deployment/cdk/opensearch-service-migration/lib/migration-services-yaml.ts @@ -1,13 +1,12 @@ -import { EngineVersion } from 'aws-cdk-lib/aws-opensearchservice'; import { ClusterAuth } from './common-utilities'; import * as yaml from 'yaml'; export class ClusterYaml { endpoint: string = ''; - version?: EngineVersion; + version?: string; auth: ClusterAuth; - constructor({endpoint, auth, version} : {endpoint: string, auth: ClusterAuth, version?: EngineVersion}) { + constructor({endpoint, auth, version} : {endpoint: string, auth: ClusterAuth, version?: string}) { this.endpoint = endpoint; this.auth = auth; this.version = version; @@ -15,6 +14,7 @@ export class ClusterYaml { toDict() { return { endpoint: this.endpoint, + version: this.version, ...this.auth.toDict(), // TODO: figure out how version should be incorporated // https://opensearch.atlassian.net/browse/MIGRATIONS-1951 @@ -108,6 +108,7 @@ export class MetadataMigrationYaml { from_snapshot: null = null; min_replicas: number = 1; otel_endpoint: string = ''; + source_cluster_version?: string; } export class MSKYaml { diff --git a/deployment/cdk/opensearch-service-migration/lib/opensearch-domain-stack.ts b/deployment/cdk/opensearch-service-migration/lib/opensearch-domain-stack.ts index eab4807df..84f9cbb8d 100644 --- a/deployment/cdk/opensearch-service-migration/lib/opensearch-domain-stack.ts +++ b/deployment/cdk/opensearch-service-migration/lib/opensearch-domain-stack.ts @@ -123,7 +123,7 @@ export class OpenSearchDomainStack extends Stack { } else { clusterAuth.noAuth = new ClusterNoAuth(); } - this.targetClusterYaml = new ClusterYaml({endpoint: `https://${domain.domainEndpoint}:443`, auth: clusterAuth, version}) + this.targetClusterYaml = new ClusterYaml({endpoint: `https://${domain.domainEndpoint}:443`, auth: clusterAuth, version: version.toString()}) } diff --git a/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-console-stack.ts b/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-console-stack.ts index 42d4a0995..96135d343 100644 --- a/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-console-stack.ts +++ b/deployment/cdk/opensearch-service-migration/lib/service-stacks/migration-console-stack.ts @@ -235,13 +235,14 @@ export class MigrationConsoleStack extends MigrationServiceCore { const getTargetSecretsPolicy = props.servicesYaml.target_cluster.auth.basicAuth?.password_from_secret_arn ? getTargetPasswordAccessPolicy(props.servicesYaml.target_cluster.auth.basicAuth?.password_from_secret_arn) : null; - const getSourceSecretsPolicy = props.servicesYaml.source_cluster?.auth.basicAuth?.password_from_secret_arn ? - getTargetPasswordAccessPolicy(props.servicesYaml.source_cluster?.auth.basicAuth?.password_from_secret_arn) : null; + const getSourceSecretsPolicy = props.sourceCluster?.auth.basicAuth?.password_from_secret_arn ? + getTargetPasswordAccessPolicy(props.sourceCluster?.auth.basicAuth?.password_from_secret_arn) : null; // Upload the services.yaml file to Parameter Store let servicesYaml = props.servicesYaml servicesYaml.source_cluster = props.sourceCluster servicesYaml.metadata_migration = new MetadataMigrationYaml(); + servicesYaml.metadata_migration.source_cluster_version = props.sourceCluster?.version if (props.otelCollectorEnabled) { const otelSidecarEndpoint = OtelCollectorSidecar.getOtelLocalhostEndpoint(); if (servicesYaml.metadata_migration) { diff --git a/deployment/cdk/opensearch-service-migration/lib/service-stacks/reindex-from-snapshot-stack.ts b/deployment/cdk/opensearch-service-migration/lib/service-stacks/reindex-from-snapshot-stack.ts index 17a6502f9..b42f3dde5 100644 --- a/deployment/cdk/opensearch-service-migration/lib/service-stacks/reindex-from-snapshot-stack.ts +++ b/deployment/cdk/opensearch-service-migration/lib/service-stacks/reindex-from-snapshot-stack.ts @@ -25,6 +25,7 @@ export interface ReindexFromSnapshotProps extends StackPropsExt { readonly extraArgs?: string, readonly otelCollectorEnabled: boolean, readonly clusterAuthDetails: ClusterAuth + readonly sourceClusterVersion?: string } export class ReindexFromSnapshotStack extends MigrationServiceCore { @@ -71,6 +72,7 @@ export class ReindexFromSnapshotStack extends MigrationServiceCore { let rfsCommand = `/rfs-app/runJavaWithClasspath.sh com.rfs.RfsMigrateDocuments --s3-local-dir /tmp/s3_files --s3-repo-uri ${s3Uri} --s3-region ${this.region} --snapshot-name rfs-snapshot --lucene-dir '/lucene' --target-host ${osClusterEndpoint}` rfsCommand = props.clusterAuthDetails.sigv4 ? rfsCommand.concat(`--target-aws-service-signing-name ${props.clusterAuthDetails.sigv4.serviceSigningName} --target-aws-region ${props.clusterAuthDetails.sigv4.region}`) : rfsCommand rfsCommand = props.otelCollectorEnabled ? rfsCommand.concat(` --otel-collector-endpoint ${OtelCollectorSidecar.getOtelLocalhostEndpoint()}`) : rfsCommand + rfsCommand = props.sourceClusterVersion ? rfsCommand.concat(` --source-version ${props.sourceClusterVersion}`) : rfsCommand rfsCommand = parseAndMergeArgs(rfsCommand, props.extraArgs); let targetUser = ""; diff --git a/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts b/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts index 2686e3be5..7451aadad 100644 --- a/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts +++ b/deployment/cdk/opensearch-service-migration/lib/stack-composer.ts @@ -487,6 +487,7 @@ export class StackComposer { vpc: networkStack.vpc, extraArgs: reindexFromSnapshotExtraArgs, clusterAuthDetails: servicesYaml.target_cluster?.auth, + sourceClusterVersion: sourceCluster?.version, stackName: `OSMigrations-${stage}-${region}-ReindexFromSnapshot`, description: "This stack contains resources to assist migrating historical data, via Reindex from Snapshot, to a target cluster", stage: stage, diff --git a/deployment/cdk/opensearch-service-migration/options.md b/deployment/cdk/opensearch-service-migration/options.md index f66f43875..17b972311 100644 --- a/deployment/cdk/opensearch-service-migration/options.md +++ b/deployment/cdk/opensearch-service-migration/options.md @@ -51,7 +51,7 @@ In all other cases, the required components of each cluster object are: The optional component is: -- `version` -- the Elasticsearch or OpenSearch version of the cluster, in the format of `OS_x.y` or `ES_x.y` +- `version` -- the Elasticsearch or OpenSearch version of the cluster, in the format of `OS_x.y` or `ES_x.y` This will be passed to the ReindexFromSnapshot service, if enabled, and provided for the metadata migration on the Migration Console. It defaults to `ES_7.10.2`. ### Reindex from Snapshot (RFS) Service Options diff --git a/docs/ClientTrafficSwinging.md b/docs/ClientTrafficSwinging.md new file mode 100644 index 000000000..66b04dcbb --- /dev/null +++ b/docs/ClientTrafficSwinging.md @@ -0,0 +1,109 @@ +# Application Load Balancer: Routing Traffic for Migration + +## What is an Application Load Balancer? + +An Application Load Balancer (ALB) is a service that manages and distributes incoming network traffic across multiple targets, such as servers or containers. + +## How Our ALB Works + +Our ALB is configured to manage traffic for different components of our migration system. Here's a straightforward explanation of its operation: + +1. **Single Entry Point**: The ALB serves as a single point of entry for all incoming requests. Users and applications only need to know one address to access our services. + +2. **Secure Communication**: All communication with the ALB is encrypted, ensuring data privacy and security during transmission over the internet. + +3. **Intelligent Routing**: The ALB uses different ports to direct traffic to various services. Each port corresponds to a specific service within our system. + +## Services and Listeners + +Our ALB is set up to route traffic to these main services: + +1. **Capture Proxy** (Port 9201): This service records incoming traffic and forwards requests to the source cluster. + +2. **Target Cluster Proxy** (Port 9202): This service forwards requests to the new cluster we're migrating to, acting as an intermediary between the ALB and the target cluster. + +3. **Weighted Proxy** (Port 9200): This forwards requests to either the Capture Proxy or Target Cluster Proxy based on configuration, see [Weighted Proxy](#weighted-proxy) + +## Weighted Proxy + +A key feature of our ALB is its ability to distribute traffic between the old and new systems. This is accomplished using the weighted listener on port `9200`: + +- The ALB can direct a portion or all of the requests to either the Capture Proxy or Target Cluster Proxy. +- We can adjust the traffic distribution between each system, allowing for a gradual transition. + +This feature, known as weighted routing, operates as follows: +- Each route (Capture Proxy and Target Cluster Proxy) is assigned a numerical weight. +- The ALB distributes traffic proportionally based on these weights. +- For example, with weights of 80 for Capture Proxy and 20 for Target Cluster Proxy, 80% of traffic would go to the source cluster and 20% to the target cluster. +- These weights can be adjusted in real-time, allowing for precise control over the migration process. + +This weighted routing is valuable because it allows us to: +- Test the new system with actual traffic while maintaining the old system's operation (careful planning is needed to prevent data loss). +- Incrementally increase traffic to the target cluster, ensuring a smooth migration (again, careful planning is needed to prevent data loss). +- Quickly modify traffic distribution or revert to the source cluster if issues arise. + +## Benefits for Your Migration + +Using this ALB setup offers several advantages: + +1. **Flexibility**: We can easily adjust traffic routing without modifying anything on the user or application side. + +2. **Safety**: The ability to split traffic enables safer, more controlled migrations. + +3. **Monitoring**: We can observe how both systems handle traffic during the migration process. And collect top level metrics from the ALB. + +4. **Seamless Transition**: Users and applications experience no disruption as traffic is shifted to the new cluster. + +5. **Smoke Testing**: Ports 9201 and 9202 can be used for preliminary testing before shifting client traffic: + - Port 9201 (Capture Proxy): Test requests can be sent here to confirm that the source cluster is still accessible and functioning correctly through the ALB. + - Port 9202 (Target Cluster Proxy): Send test requests to this port to verify that the new target cluster is properly set up and responding as expected. + - By testing both ports, we can ensure that both the old and new systems are working correctly before gradually shifting real client traffic using the weighted routing on port 9200. + + This smoke testing process allows us to: + - Verify the ALB configuration is correct for both source and target clusters. + - Identify any potential issues with either cluster before exposing them to real client traffic. + - Gain confidence in the migration setup before initiating the actual traffic shift. + +By utilizing the ALB in this manner, we can ensure a smooth, controlled, and safe migration process, minimizing risks and disruptions to your service. The ability to perform smoke tests on specific ports adds an extra layer of verification and safety to the migration process. + +### Architecture Diagram + +```mermaid +graph TD + subgraph ExistingClientNetworking + Client[Client] + NLB[Existing Network Load Balancer] + end + + subgraph MigrationAssistantInfrastructure + ALB[Application Load Balancer] + subgraph ALB + L1[Weighted Listener :9200] + L2[Source Listener :9201] + L3[Target Listener :9202] + end + CP[Capture Proxy] + TCP[Target Cluster Proxy] + end + + SC[Source Cluster] + TC[Target Cluster] + + + %% Invisible lines for correct rendering order + ExistingClientNetworking ~~~ MigrationAssistantInfrastructure + NLB ~~~ ALB + + %% Connections + Client --> NLB + NLB --> L1 + Client -.->|Optional Direct Routing| L1 + + L1 -.->|Weight X%| CP + L1 ==>|Weight Y%| TCP + L2 --> CP + L3 --> TCP + + CP --> SC + TCP --> TC +``` diff --git a/docs/TrafficCaptureAndReplayDesign.md b/docs/TrafficCaptureAndReplayDesign.md new file mode 100644 index 000000000..02eeafa03 --- /dev/null +++ b/docs/TrafficCaptureAndReplayDesign.md @@ -0,0 +1,271 @@ +# Traffic Capture and Replay + +## Overview + +Two main components support cluster mirroring. The first component is the Capture Proxy, which relays network traffic +from HTTP requests to a source cluster into a durable, scalable stream. The second component, the Traffic Replayer, +replicates the traffic observed by the Proxy onto a target cluster. In this case mirroring does three things. + +1. Illustrates differences in behavior between the source and target clusters. +2. Stresses the target cluster very similarly to the source cluster. +3. It keeps a target’s documents and metadata in sync with a source cluster. + +Data is buffered through Kafka from the proxy to the replayer. The components also send metrics and traces to an +otel-collector, which is deployed as a sidecar, which in turn publishes instrumentation. + +Here are the main steps to synchronize a target cluster from a source cluster: + +1. Traffic is directed to the existing cluster, reaching each coordinator node. +2. A Capture Proxy is added in front of the coordinator nodes in the cluster, allowing for traffic capture and storage. + (see [here](./ClientTrafficSwinging.md) for details about how to use an ALB to do this). +3. A historical backfill is triggered to synchronize the documents in the target from the source as it was at some + point in time. That point in time will/must be after all traffic has been captured. +4. Following the backfill, the Traffic Replayer begins replaying the captured traffic to the target cluster. +5. The user evaluates the differences between source and target responses. +6. After confirming that the new cluster's functionality meets expectations, the target server is ready to become the + new cluster. Note that the target continues to be synchronized from the replayer. If customers are especially + concerned about greater durations where results may be inconsistent due to lag between the source cluster and the + target cluster (ideally this is around 1s), the capture proxy could reject modifications, forcing clients to + resend shortly, allowing the target cluster to pickup those modifications as the target fleet replaces the source + fleet. + +## Capture Proxy + +The Capture Proxy terminates TLS and replicates the decrypted read/writes streams as they arrive to Kafka. Since the +Capture Proxy is handling data on the critical path for the source cluster, the proxy is designed to offload data as +efficiently as possible to minimize the proxy’s impact on overall performance (latency, load, etc). The Capture Proxy +parses its TLS configuration the same way as OpenSearch, from a yaml config, with the same keys. + +The proxy is expected to supplant the original source cluster endpoint so that clients can continue to operate without +any changes. One way to accomplish that is to install a proxy alongside the source cluster’s coordinating nodes and +shift the coordinating nodes’ configuration to use a port bound only to the loopback address and likely without TLS, +as encrypting local traffic with TLS is expensive and unnecessary. Another approach is +described [here](./ClientTrafficSwinging.md). + +The proxy can also be deployed on standalone hardware. However, two caveats remain. + +* The proxy is only designed to proxy traffic for a single destination. If that destination is a large number of nodes + with a load balancer, any number of proxies that are necessary to support the traffic load can be setup and all will + send traffic though the nodes that the load balancer is using. +* The second caveat to installing the proxy on separate hardware is that the infrastructure will need to change and + traffic will need to be routed exclusively through the proxy, which itself is more infrastructure to change. This will + also increase latency for all traffic over a colocated solution. + +### TLS + +In order for the proxy to write data that can be replayed and used for comparison, the request and response data must be +stored so that HTTP messages can be reconstructed at a later point in time. If an existing client and server (cluster) +are using TLS to transfer data, that data will first be decrypted before being offloaded. When using Amazon Managed +Streaming for Apache Kafka, AWS Authentication is used, data is sent via TLS, and it is stored in an encrypted format. + +### Additional Impact + +In addition to the impact incurred from TLS decrypting and encrypting, there may be a significant impact to network +load. This solution assumes that the network has enough capacity to double the network traffic, albeit to different +destinations. The proxy doesn’t compress traffic because many requests and responses may already be compressed. + +If a PUT or any other mutating call is dropped from the replay, it could have a long-lasting and irreversible impact on +all future results. Because of that, the Capture Proxy parses the incoming stream as HTTP messages to determine the +importance of an HTTP request. All GET traffic is immediately forwarded to the source cluster while data is +asynchronously offloaded to Kafka. Mutating requests such as PUT, POST, DELETE, PATCH etc are handled more carefully. +The Capture Proxy makes certain that all mutating requests have been committed to Kafka before the request is fully +‘released’ and sent to the source cluster. This behavior means that GET traffic should flow through the system without +being impacted by the latency of calls to Kafka. However, mutating requests will be impacted. Clients that have made +those requests will not receive a response or will not be able to make another request until all prior offloaded traffic +for the connection has been committed (which could include offloading previous GET requests that had been sent on the +same connection). + +That guarantees that no mutating request was sent to the source without first being committed to Kafka. However, it also +means that a request could be committed to Kafka without ever being sent and handled by the downstream service. Requests +that are suspected of not being processed (or fully processed) by the source cluster are detectable by the Capture +Proxy. Those requests will be missing a response. Notice that since responses themselves may not be fully returned in +every case that its request is handled, there may be other cases where a mutating request DID succeed on the source +cluster but no response is present. The Capture Proxy doesn’t yet reconcile the which of these requests have likely +succeeded and which have failed. However, in practice, many real-world examples would have retried the failed request, +resulting in a received response. + +All network activity is asynchronously data-driven, using the same framework (netty) that Elasticsearch, OpenSearch, and +many other projects use. Using that same framework also mitigates some risk that HTTP could be parsed differently by the +source and the proxy. + +### Protocol + +Captured data is organized into TrafficObservations (Read, Write, Close, etc) that have timestamps and are organized +into larger “TrafficStream” objects which are written as records to Kafka. These observations are serialized +as [Protobuf](../TrafficCapture/captureProtobufs/src/main/proto/TrafficCaptureStream.proto) wrappers to the raw bytes +that were received or sent by the Proxy sans TLS. TrafficStream objects are organized by connection, with each socket +connection represented by a sequences of TrafficStreams, which will have TrafficObservations for that connection only. +Those TrafficStreams are flushed to Kafka after buffering or after a mutating request has been received. Concurrent +connections will have TrafficStream objects that are interleaved with each other. Each TrafficStream will have its own +respective connectionId, which is a globally unique id for that connection. A unique nodeId is also included for +diagnostics and future Kafka partitioning. + +## Traffic Replayer + +The Traffic Replayer parses the Protobuf encoded objects recorded by the proxy; reconstructs them into HTTP requests; +sends them to the target cluster; and records the responses alongside the request and the traffic from the original +source interaction. + +The Traffic Replayer must group TrafficStreams by their connectionIds into reconstituted TCP streams. Individual +requests are parsed and sent through a processing pipeline that rewrites the request as necessary, then schedules and +sends the requests to match the source time intervals. Responses are aggregated along with the request and source +messages. Following that, the TrafficStream objects are committed from the Kafka topic so that they are not processed +again by a future or concurrently running Replayer. + +### Message Transformation + +The reassembly process of the captured traffic is careful to preserve timestamps of the originally observed traffic. +Once all the bytes for a request have been accumulated, the bytes are sent through a netty pipeline for transformation. +As per configuration, this processing may include rewriting headers, such as the Host value, changing +User-Authentication, and transforming the contents of the payload. The TrafficReplayer parses the original source +request traffic stream into a Map object with the headers and payload of the original message in a json-friendly +key-value structure. That Map object is passed through an IJsonTransformer object that may rewrite the request by +altering headers or the body. To minimize unnecessary and expensive operations, the netty pipeline parses the HTTP +headers first and runs the transformation before the pipeline has been fully configured. If based upon the headers only, +the transformation did not attempt to access the payload, the pipeline won’t be configured to parse the json from the +body from the message. The pipeline will attempt to setup as few handlers as possible to eliminate unnecessary (de) +compression and repackaging. + +When the initial transformation has attempted to access the payload, the entire message needs to be transformed. In that +case, netty handlers are attached to do all of the work required to parse an HTTP payload into a json-like Map (HTTP +chunking, decompression, json parsing, followed by compression, etc). Generally, the handlers stream data as much as +possible so that efficiency can be maximized. + +At the end of the transformation pipeline, a new sequence of network buffers has been formatted and is ready to be sent +to the target server. The shape and pacing of the sequence of buffers should closely match that of the original +sequence. In other words, if the source got 120 bytes with one byte per second, the target request will also get 120 +parts over 120 seconds. + +In some cases, the pipeline may be unable to parse a message, or the message might not require any rewrite. In those +cases, the parsing of the current request is unwound and the request is sent exactly as sent to the source to the target +cluster. The response will be handled like any response for a fully transformed message, though the final metadata will +show whether the request had transformation skipped or if it was due to an error. + +This message transformation also includes rewriting authorization headers. In the Basic-Auth case, that rewrite will +only involve the headers. If there were no other transformations, the body of the content will not need to be parsed. +However, if the authorization scheme being used is AWS Auth (SigV4), a handler to parse the body will be added to the +pipeline alongside mechanisms to fully consume the contents so that the signature can be accurately computed. + +#### Auth Caveat + +The Replayer doesn’t have a way to determine the validity of the incoming messages. It doesn’t have the HTTP basic-auth +passwords, nor does it have access to the public keys used by SigV4. This creates a significant security issue that +currently diminishes the value of auth for OpenSearch clusters. Currently, all requests, regardless of whether they were +validly or maliciously signed will be rewritten with the same auth headers as per the configuration. We can’t leverage +the responses to determine validity since there will be some changes that the replayer must run even though there was no +response present. + +#### User Transformations + +Users may specify what Transformation to run by providing a jar file that can load an implemention of the +IJsonTransformer class via Java’s ServiceLoader. As described in Message Transformation section, the complexities of +parsing HTTP messages are abstracted away. A transformer can switch off of URI paths, headers, and run sophisticated +json remapping by pulling in libraries such as Jackson, GSON, or json manipulation packages like Jolt or JMESPath. As +progress continues, additional transformations will be developed to accomplish the required transformations between +versions, plugins, etc more easily. + +### Timing + +The Replayer may be started long after the Capture Proxy has begun recording traffic. Recall that the Replayer tries to +match the client request traffic exactly as it was received by the source cluster so that when comparing results, both +clusters, for any request, the clusters would have been undergoing the same stresses. To do that, the Traffic Replayer +manages its own sense of time to send requests to the target. It shifts the original requests’ timestamps uniformly so +that delays between each observation and request can also be preserved. + +A replay will generally start at the beginning of the captured stream and it will fix the current time to the time of +the first interaction. For users that would like to catch-up or stress test the system, the Replayer’s time mapping +function can include a speedup factor (F) so that something that happened N seconds after the initially observation will +be scheduled N/F seconds after the Replayer has started. This functionality is managed by a TimeShifter class that is +effectively just a function that maps scalar values after some initialization. Timing values can be controlled via +command line parameters. + +That timing drives much of the rest of the TrafficReplayer. When a request is fully reconstructed, the message +transformation work is *scheduled* to be done just before it would be scheduled to be sent. That’s to guarantee that +temporally sensitive values like SigV4 signatures won’t go stale. It also keeps more data within the same thread, making +for less contention (cache invalidations) and allows for simpler code. + +### Sending Requests + +Like the Capture Proxy and the transformation pipelines described above, requests are sent via netty. Netty’s +architecture allows for a large number of requests to be handled concurrently through cooperative multitasking. While +that requires code to be data-driven and to never block, it also affords the use of simple data structures that need not +be thread-safe across multiple threads. Several classes are designed to only run from a single thread. Those will be +initialized for each worker thread that netty spins up (which can be specified on the command line). + +Netty manages connections to the target cluster within its own EventLoops. An EventLoop within the TrafficReplayer will +have a connection to the target service, which may break and need to be reestablished. However, that same EventLoop (and +its Thread) will be affiliated with its connection for the lifetime of the connection. That lifetime will be terminated +either when the connection’s TrafficStream has encountered a “Close” observation OR if no observations have been +encountered in a period of time and they are expired by the accumulation phase. + +Each connection’s EventLoop and Channel are grouped within a ConnectionReplaySession. This session object also includes +data to schedule the interactions for transforming requests, sending them, and closing connections. The schedules are +maintained as a key-value map from the time that an operation should happen, post time shifting (so in real-time, not +source time). As work items are completed for a connection’s session, the next item is pulled from the schedule if it is +ready to run or a timer is set on the EventLoop to rerun it when appropriate. Because there are multiple interactions +scheduled within each connection’s session and each is run sequentially and exclusively, the actual times that +interactions occur could drift. For example, assume the source service took 5 seconds to service each of 6 requests +sequentially on one socket connection that was kept alive. If the target service takes 10 seconds to service each +request, the target will take 60 seconds to run those 6 interactions instead of 30 from the source cluster. + +However, if a source cluster had the same interactions and latencies for 6 requests BUT sent them on separate +connections, the total time to send all the requests to the target service could be 35 seconds, since requests would +overlap by 5 seconds. Currently, schedules are isolated to a given connection. Since a connection is defined by a +sequence of requests, we must wait for the previous requests to finish before proceeding. Without the connection +isolation though, requests will be sent without those constraints. +***This does create a caveat that if the target cluster cannot keep up with the pacing of the source cluster, the +Replayer may not be able to match the timing patterns that the source cluster experienced.*** + +### Throttling + +The Traffic Replayer has a TrafficCaptureSource that it uses as an abstraction over a Kafka topic. If the Kafka topic +has days of data, it’s critical that the Replayer consume and hold only the data that is necessary to effectively send +the requests as per the above constraints. The Replayer needs to throttle its Kafka consumption so that it can keep its +memory footprint manageable. To figure out what’s necessary, the Replayer needs to consume enough data so that it can +reconstruct the TrafficStreams into requests. + +Within a user-specified window, we expect to either accumulate more observations for a TrafficStream or expire it and +give up. That prevents leaking memory within the Replayer when Proxy’s had faults that prevented them from sending close +events for streams. That limit is independent of how far ahead in the recorded stream should the Replayer advance to +while consuming input. Setting that backpressure ‘lookahead’ limit to a multiple of the expiration window provides a +backpressure to limit memory. However, those two limits still don'’t put a total bound on the peak amount of memory +required for the Replayer. Such a bound would be problematic as there could be an unlimited number of simultaneous +connections with ongoing traffic. If a bound were enforced, progress may not be made on accumulating all of the +requests, trading an Out-of-Memory situation with a deadlocked replayer. The right solution in those cases is to scale +the Replayer to use more memory, either vertically or horizontally. Right-sizing in these cases will be an exercise in +understanding peak load to the source cluster and/or trial and error for provisioning the Traffic Replayer. + +The TrafficReplayer may try to send too many requests simultaneously. This will happen if there aren’t enough ports +available on fleet of replayers to support the number of simultaneous connections in use. This can occur when the number +of Replayers is much smaller than the proxy fleet or if the time speedup factor was high on a modestly sized cluster. In +these cases, an additional throttling mechanism is required to restrict how many simultaneous connections can be made. A +command line option is available to limit how many requests can be in-progress at any point in time. When a request has +been reconstructed, if the Replayer has already saturated the total number of requests that it can handle, the new +request will be blocked from sending until other requests have finished. Notice that this is currently bound to the +number of requests, not connections, though a connection will only handle only one request at a time, so this is a rough +substitute for the number of connections. + +For customers that need to achieve higher throughputs, they can scale the solution horizontally. + +### Horizontal Scaling + +The Capture Proxy writes observations to a Kafka topic. Given that Kafka can accept writes from many clients at high +throughputs, the Proxy can easily scale to any number of machines. Like the Proxy, the Replayer simply consumes from +Kafka and Kafka can support many consumers. Since connections are the lowest level atomic groupings and a single client +and server at a prior point in time were already handling that connection, by scaling the Kafka topic for the +appropriate number of Proxies and Replayers, we can handle arbitrarily large loads. + +However, the scaling factor for Kafka is the number of partitions. Those partitions will need to be setup to key off of +the nodeId or the connectionId. The partition count itself will need to be set at the time that a topic is created so +that traffic for a given connection never gets remapped while a connection’s observations are being written to the +topic. + +### Outputting Results + +The Traffic Replayer uses log4j2 for its application logs. It also uses Log4J2 to output some of its other output, +including logs destined for metrics and the results of the source and target traffic interactions. Care should be taken +in adjusting the log4j2.properties files, when necessary. + +The results, which are logged not just through log4j2 but also to a file, which is provided by a command line parameter. +This result output will be a stream of json formatted objects with the source/target requests/responses. Those will +include headers, timestamps, and the full bodies base64 encoded. diff --git a/gradle.properties b/gradle.properties index 131ae1c2b..dd1bb1d07 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,2 +1,6 @@ org.gradle.caching=true +org.gradle.configuration-cache=true +# Set Gradle Daemon's idle timeout to 30 minutes +org.gradle.daemon.idletimeout=1800000 org.gradle.jvmargs=-Xmx2g -XX:MaxMetaspaceSize=512m -XX:+HeapDumpOnOutOfMemoryError -Dfile.encoding=UTF-8 +org.gradle.parallel=true diff --git a/testHelperFixtures/build.gradle b/testHelperFixtures/build.gradle index 70b0fd822..e5b3ab7ed 100644 --- a/testHelperFixtures/build.gradle +++ b/testHelperFixtures/build.gradle @@ -35,6 +35,7 @@ dependencies { testFixturesImplementation group: 'org.bouncycastle', name: 'bcprov-jdk18on' testFixturesImplementation group: 'org.bouncycastle', name: 'bcpkix-jdk18on' testFixturesImplementation group: 'org.junit.jupiter', name: 'junit-jupiter-api' + testFixturesImplementation group: 'org.hamcrest', name: 'hamcrest' testFixturesImplementation group: 'org.slf4j', name: 'slf4j-api' testFixturesApi group: 'org.testcontainers', name: 'testcontainers' testFixturesApi group: 'org.testcontainers', name: 'toxiproxy' diff --git a/testHelperFixtures/src/test/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtensionTest.java b/testHelperFixtures/src/test/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtensionTest.java new file mode 100644 index 000000000..30d301ec4 --- /dev/null +++ b/testHelperFixtures/src/test/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtensionTest.java @@ -0,0 +1,64 @@ +package org.opensearch.migrations.testutils; + +import java.time.Duration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.parallel.Execution; +import org.junit.jupiter.api.parallel.ExecutionMode; + + +@Execution(ExecutionMode.SAME_THREAD) +class NettyLeakCheckTestExtensionTest { + + String testCase; + int counter; + long startTimeNanos; + + @BeforeEach + public void beforeTest() { + testCase = null; + counter = 0; + startTimeNanos = System.nanoTime(); + } + + @AfterEach + public void afterTest() { + var observedTestDuration = Duration.ofNanos(System.nanoTime()-startTimeNanos); + switch (testCase) { + case "testMaxTimeSupercedesReps": + Assertions.assertTrue(counter < 20, "counter=" + counter); + Assertions.assertTrue(Duration.ofMillis(100).minus(observedTestDuration).isNegative()); + break; + case "testMinTimeSupercedesReps": + Assertions.assertTrue(counter > 1, "counter=" + counter); + Assertions.assertTrue(Duration.ofMillis(100).minus(observedTestDuration).isNegative()); + break; + default: + Assertions.fail("unknown test case: " + testCase); + } + } + + @Test + @WrapWithNettyLeakDetection(maxRuntimeMillis = 100, repetitions = 20) + public void testMaxTimeSupercedesReps() throws Exception { + testCase = getMyMethodName(); + ++counter; + Thread.sleep(10); + } + + @Test + @WrapWithNettyLeakDetection(minRuntimeMillis = 100, repetitions = 1) + public void testMinTimeSupercedesReps() throws Exception { + testCase = getMyMethodName(); + ++counter; + Thread.sleep(10); + } + + private static String getMyMethodName() { + var backtrace = Thread.currentThread().getStackTrace(); + return backtrace[2].getMethodName(); + } +} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/ContainsStringCount.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/ContainsStringCount.java new file mode 100644 index 000000000..0cec5c714 --- /dev/null +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/ContainsStringCount.java @@ -0,0 +1,35 @@ +package org.opensearch.migrations.matchers; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; + +import lombok.AllArgsConstructor; + +@AllArgsConstructor +public class ContainsStringCount extends TypeSafeMatcher { + private final String expectedString; + private final int expectedCount; + + @Override + public void describeTo(Description description) { + description.appendText("a string containing '" + expectedString + "' " + expectedCount + " times"); + } + + @Override + protected void describeMismatchSafely(String item, Description mismatchDescription) { + mismatchDescription.appendText("was found " + containsStringCount(item) + " times"); + } + + @Override + protected boolean matchesSafely(String item) { + return containsStringCount(item) == expectedCount; + } + + private int containsStringCount(String item) { + return item == null ? 0 : item.split(expectedString, -1).length - 1; + } + + public static ContainsStringCount containsStringCount(String s, int n) { + return new ContainsStringCount(s, n); + } +} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/HasLineCount.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/HasLineCount.java new file mode 100644 index 000000000..09203d8b1 --- /dev/null +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/matchers/HasLineCount.java @@ -0,0 +1,34 @@ +package org.opensearch.migrations.matchers; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; + +import lombok.AllArgsConstructor; + +@AllArgsConstructor +public class HasLineCount extends TypeSafeMatcher { + private int expectedLineCount; + + @Override + public void describeTo(Description description) { + description.appendText("a string with " + expectedLineCount + " lines"); + } + + @Override + protected void describeMismatchSafely(String item, Description mismatchDescription) { + mismatchDescription.appendText("was a string with " + item.split(System.lineSeparator()).length + " lines"); + } + + @Override + protected boolean matchesSafely(String item) { + return newlineCount(item) == expectedLineCount; + } + + private int newlineCount(String item) { + return item == null ? 0 : item.split("\n").length; + } + + public static HasLineCount hasLineCount(int n) { + return new HasLineCount(n); + } +} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpFirstLine.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpFirstLine.java deleted file mode 100644 index 7a6ad26d9..000000000 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpFirstLine.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.opensearch.migrations.testutils; - -import java.net.URI; - -public interface HttpFirstLine { - String verb(); - - URI path(); - - String version(); -} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequest.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequest.java new file mode 100644 index 000000000..de487e162 --- /dev/null +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequest.java @@ -0,0 +1,15 @@ +package org.opensearch.migrations.testutils; + +import java.net.URI; +import java.util.List; +import java.util.Map; + +public interface HttpRequest { + String getVerb(); + + URI getPath(); + + String getVersion(); + + List> getHeaders(); +} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequestFirstLine.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequestFirstLine.java deleted file mode 100644 index fc944723d..000000000 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/HttpRequestFirstLine.java +++ /dev/null @@ -1,11 +0,0 @@ -package org.opensearch.migrations.testutils; - -import java.net.URI; - -public interface HttpRequestFirstLine { - String verb(); - - URI path(); - - String version(); -} diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtension.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtension.java index 1dadf85bd..ac61e0fbd 100644 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtension.java +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/NettyLeakCheckTestExtension.java @@ -12,6 +12,7 @@ import lombok.Lombok; public class NettyLeakCheckTestExtension implements InvocationInterceptor { + public static final int DEFAULT_NUM_REPETITIONS = 16; private final boolean allLeakChecksAreDisabled; public NettyLeakCheckTestExtension() { @@ -23,19 +24,49 @@ private void wrapWithLeakChecks( Callable repeatCall, Callable finalCall ) throws Throwable { - if (allLeakChecksAreDisabled || getAnnotation(extensionContext).map(a -> a.disableLeakChecks()).orElse(false)) { + if (allLeakChecksAreDisabled || getAnnotation(extensionContext).map(WrapWithNettyLeakDetection::disableLeakChecks).orElse(false)) { CountingNettyResourceLeakDetector.deactivate(); finalCall.call(); return; } else { CountingNettyResourceLeakDetector.activate(); - int repetitions = getAnnotation(extensionContext).map(a -> a.repetitions()) + var repetitions = getAnnotation(extensionContext).map(WrapWithNettyLeakDetection::repetitions) .orElseThrow(() -> new IllegalStateException("No test method present")); + var minRuntimeMs = getAnnotation(extensionContext).map(WrapWithNettyLeakDetection::minRuntimeMillis) + .orElseThrow(() -> new IllegalStateException("No test method present")); + var maxRuntimeMs = getAnnotation(extensionContext).map(WrapWithNettyLeakDetection::maxRuntimeMillis) + .orElseThrow(() -> new IllegalStateException("No test method present")); + if (repetitions == -1 && + minRuntimeMs == -1 && + maxRuntimeMs == -1) { + repetitions = DEFAULT_NUM_REPETITIONS; + } + assert minRuntimeMs <= 0 || maxRuntimeMs <= 0 || minRuntimeMs <= maxRuntimeMs : + "expected maxRuntime to be >= minRuntime"; - for (int i = 0; i < repetitions; i++) { - ((i == repetitions - 1) ? finalCall : repeatCall).call(); + long nanosSpent = 0; + for (int runNumber = 1; ; runNumber++) { + var startTimeNanos = System.nanoTime(); + boolean lastRun = false; + { + var timeSpentMs = nanosSpent / (1000 * 1000); + if (repetitions >= 0) { + lastRun = runNumber >= repetitions; + } + if (minRuntimeMs > 0) { + lastRun = timeSpentMs >= minRuntimeMs; + } + if (maxRuntimeMs > 0 && !lastRun) { + lastRun = timeSpentMs >= maxRuntimeMs; + } + } + (lastRun ? finalCall : repeatCall).call(); + nanosSpent += (System.nanoTime() - startTimeNanos); System.gc(); System.runFinalization(); + if (lastRun) { + break; + } } Assertions.assertEquals(0, CountingNettyResourceLeakDetector.getNumLeaks()); @@ -76,11 +107,9 @@ public void interceptTestTemplateMethod( var selfInstance = invocationContext.getTarget() .orElseThrow(() -> new IllegalStateException("Target instance not found")); wrapWithLeakChecks(extensionContext, () -> { - { - Method m = invocationContext.getExecutable(); - m.setAccessible(true); - return m.invoke(selfInstance, invocationContext.getArguments().toArray()); - } + Method m = invocationContext.getExecutable(); + m.setAccessible(true); + return m.invoke(selfInstance, invocationContext.getArguments().toArray()); }, () -> wrapProceed(invocation)); } diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleHttpServer.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleHttpServer.java index 5682ec71c..71c5b47d4 100644 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleHttpServer.java +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleHttpServer.java @@ -3,13 +3,19 @@ import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; +import java.util.AbstractMap; +import java.util.List; +import java.util.Map; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; +import java.util.stream.Collectors; import com.sun.net.httpserver.HttpServer; import com.sun.net.httpserver.HttpsConfigurator; import com.sun.net.httpserver.HttpsParameters; import com.sun.net.httpserver.HttpsServer; +import lombok.AllArgsConstructor; +import lombok.Getter; import lombok.Lombok; /** @@ -26,7 +32,7 @@ public class SimpleHttpServer implements AutoCloseable { public static SimpleHttpServer makeServer( boolean useTls, - Function makeContext + Function makeContext ) throws PortFinder.ExceededMaxPortAssigmentAttemptException { var testServerRef = new AtomicReference(); PortFinder.retryWithNewPortUntilNoThrow(port -> { @@ -39,31 +45,13 @@ public static SimpleHttpServer makeServer( return testServerRef.get(); } - public static class PojoHttpRequestFirstLine implements HttpRequestFirstLine { + @Getter + @AllArgsConstructor + public static class PojoHttpRequest implements HttpRequest { private final String verb; private final URI path; private final String version; - - public PojoHttpRequestFirstLine(String verb, URI path, String version) { - this.verb = verb; - this.path = path; - this.version = version; - } - - @Override - public String verb() { - return verb; - } - - @Override - public URI path() { - return path; - } - - @Override - public String version() { - return version; - } + private final List> headers; } private static HttpsServer createSecureServer(InetSocketAddress address) throws Exception { @@ -93,18 +81,21 @@ public void configure(HttpsParameters params) { * @param port * @return the port upon successfully binding the server */ - public SimpleHttpServer(boolean useTls, int port, Function contentMapper) + public SimpleHttpServer(boolean useTls, int port, Function contentMapper) throws Exception { var addr = new InetSocketAddress(LOCALHOST, port); this.useTls = useTls; httpServer = useTls ? createSecureServer(addr) : HttpServer.create(addr, 0); httpServer.createContext("/", httpExchange -> { - var requestToMatch = new PojoHttpRequestFirstLine( + var requestToMatch = new PojoHttpRequest( httpExchange.getRequestMethod(), httpExchange.getRequestURI(), - httpExchange.getProtocol() - ); + httpExchange.getProtocol(), + httpExchange.getRequestHeaders().entrySet().stream() + .flatMap(keyValueList -> keyValueList.getValue().stream() + .map(v -> new AbstractMap.SimpleEntry<>(keyValueList.getKey(), v))) + .collect(Collectors.toList())); var headersAndPayload = contentMapper.apply(requestToMatch); var responseHeaders = httpExchange.getResponseHeaders(); for (var kvp : headersAndPayload.headers.entrySet()) { diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleNettyHttpServer.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleNettyHttpServer.java index f0ce2028a..41c2fc260 100644 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleNettyHttpServer.java +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/SimpleNettyHttpServer.java @@ -3,6 +3,7 @@ import java.net.URI; import java.net.URISyntaxException; import java.time.Duration; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -29,6 +30,7 @@ import io.netty.handler.codec.http.HttpResponseEncoder; import io.netty.handler.codec.http.HttpResponseStatus; import io.netty.handler.codec.http.HttpVersion; +import io.netty.handler.logging.LoggingHandler; import io.netty.handler.ssl.SslHandler; import io.netty.handler.timeout.ReadTimeoutHandler; import io.netty.util.concurrent.DefaultThreadFactory; @@ -56,15 +58,30 @@ public class SimpleNettyHttpServer implements AutoCloseable { public static SimpleNettyHttpServer makeServer( boolean useTls, - Function makeContext + Function makeContext ) throws PortFinder.ExceededMaxPortAssigmentAttemptException { - return makeServer(useTls, null, makeContext); + return makeNettyServer(useTls, null, r -> makeContext.apply(new RequestToAdapter(r))); + } + + public static SimpleNettyHttpServer makeNettyServer( + boolean useTls, + Function makeContext + ) throws PortFinder.ExceededMaxPortAssigmentAttemptException { + return makeNettyServer(useTls, null, makeContext); } public static SimpleNettyHttpServer makeServer( boolean useTls, Duration readTimeout, - Function makeContext + Function makeContext + ) throws PortFinder.ExceededMaxPortAssigmentAttemptException { + return makeNettyServer(useTls, readTimeout, r -> makeContext.apply(new RequestToAdapter(r))); + } + + public static SimpleNettyHttpServer makeNettyServer( + boolean useTls, + Duration readTimeout, + Function makeContext ) throws PortFinder.ExceededMaxPortAssigmentAttemptException { var testServerRef = new AtomicReference(); PortFinder.retryWithNewPortUntilNoThrow(port -> { @@ -77,28 +94,31 @@ public static SimpleNettyHttpServer makeServer( return testServerRef.get(); } - private static class RequestToFirstLineAdapter implements HttpRequestFirstLine { + public static class RequestToAdapter implements HttpRequest { private final FullHttpRequest request; - public RequestToFirstLineAdapter(FullHttpRequest request) { + public RequestToAdapter(FullHttpRequest request) { this.request = request; } @Override - public String verb() { + public String getVerb() { return request.method().toString(); } @SneakyThrows @Override - public URI path() { + public URI getPath() { return new URI(request.uri()); } @Override - public String version() { + public String getVersion() { return request.protocolVersion().text(); } + + @Override + public List> getHeaders() { return request.headers().entries(); } } HttpHeaders convertHeaders(Map headers) { @@ -108,8 +128,13 @@ HttpHeaders convertHeaders(Map headers) { } private SimpleChannelInboundHandler makeHandlerFromResponseContext( - Function responseBuilder - ) { + Function responseBuilder) { + return makeHandlerFromNettyResponseContext(r -> responseBuilder.apply(new RequestToAdapter(r))); + } + + private SimpleChannelInboundHandler makeHandlerFromNettyResponseContext( + Function responseBuilder) + { return new SimpleChannelInboundHandler<>() { @Override protected void channelRead0(ChannelHandlerContext ctx, FullHttpRequest req) { @@ -118,7 +143,7 @@ protected void channelRead0(ChannelHandlerContext ctx, FullHttpRequest req) { ctx.close(); return; } - var specifiedResponse = responseBuilder.apply(new RequestToFirstLineAdapter(req)); + var specifiedResponse = responseBuilder.apply(req); var fullResponse = new DefaultFullHttpResponse( HttpVersion.HTTP_1_1, HttpResponseStatus.valueOf(specifiedResponse.statusCode, specifiedResponse.statusText), @@ -146,7 +171,7 @@ protected void channelRead0(ChannelHandlerContext ctx, FullHttpRequest req) { boolean useTLS, int port, Duration timeout, - Function responseBuilder + Function responseBuilder ) throws Exception { this.useTls = useTLS; this.port = port; @@ -168,10 +193,13 @@ protected void initChannel(SocketChannel ch) { if (timeout != null) { pipeline.addLast(new ReadTimeoutHandler(timeout.toMillis(), TimeUnit.MILLISECONDS)); } + pipeline.addLast(new LoggingHandler("A")); pipeline.addLast(new HttpRequestDecoder()); + pipeline.addLast(new LoggingHandler("B")); pipeline.addLast(new HttpObjectAggregator(16 * 1024)); + pipeline.addLast(new LoggingHandler("C")); pipeline.addLast(new HttpResponseEncoder()); - pipeline.addLast(makeHandlerFromResponseContext(responseBuilder)); + pipeline.addLast(makeHandlerFromNettyResponseContext(responseBuilder)); } }); serverChannel = b.bind(port).sync().channel(); diff --git a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/WrapWithNettyLeakDetection.java b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/WrapWithNettyLeakDetection.java index fd8a9bc8e..2e2a2e181 100644 --- a/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/WrapWithNettyLeakDetection.java +++ b/testHelperFixtures/src/testFixtures/java/org/opensearch/migrations/testutils/WrapWithNettyLeakDetection.java @@ -7,19 +7,43 @@ import org.junit.jupiter.api.extension.ExtendWith; +/** + * This annotation causes a test to be run within the NettyLeakCheckTestExtension wrapper. + * That will run a test multiple times with a CountingNettyResourceLeakDetector set as the + * ByteBuf allocator to detect memory leaks.

+ * + * Some leaks might need to put a bit more stress on the GC for objects to get cleared out + * and trigger potential checks within any resource finalizers to determine if there have + * been leaks. This could also be used to make leaks more obvious as the test environment + * itself will have many resources and looking for just one rogue ByteBufHolder in the hmap + * file could be difficult.

+ * + * In case min/max values for repetitions and runtime contradict each other, the test will + * run enough times to meet the minimum requirements even if the max repetitions or runtime + * is surpassed. + */ @Target({ ElementType.TYPE, ElementType.METHOD }) @Retention(RetentionPolicy.RUNTIME) @ExtendWith(NettyLeakCheckTestExtension.class) public @interface WrapWithNettyLeakDetection { /** - * Some leaks might need to put a bit more stress on the GC for objects to get cleared out - * and trigger potential checks within any resource finalizers to determine if there have - * been leaks. This could also be used to make leaks more obvious as the test environment - * itself will have many resources and looking for just one rogue ByteBufHolder in an hmap - * file could be difficult. - * @return + * How many repetitions the test should run, provided that it hasn't gone over the maxRuntime (if specified) + * and has run enough times to meet the minRuntime (if specified) */ - int repetitions() default 16; + int repetitions() default -1; + + /** + * Like repetitions this is a guesstimate to be provided to make sure that a test will + * put enough ByteBuf pressure and activity to trigger exceptions and be useful in dumps. + * This may take precedence over a repetitions value that is otherwise too small. + */ + long minRuntimeMillis() default -1; + /** + * Like repetitions this is a guesstimate to be provided to make sure that a test will + * put enough ByteBuf pressure and activity to trigger exceptions and be useful in dumps. + * This may take precedence over a repetitions value that is too large. + */ + long maxRuntimeMillis() default -1; /** * Set this to true to disable running any netty leak checks. This will cause the test to be diff --git a/transformation/src/main/java/org/opensearch/migrations/Flavor.java b/transformation/src/main/java/org/opensearch/migrations/Flavor.java index 151d1b7fa..f835c92dc 100644 --- a/transformation/src/main/java/org/opensearch/migrations/Flavor.java +++ b/transformation/src/main/java/org/opensearch/migrations/Flavor.java @@ -6,8 +6,8 @@ @RequiredArgsConstructor @Getter public enum Flavor { - Elasticsearch("ES"), - OpenSearch("OS"); + ELASTICSEARCH("ES"), + OPENSEARCH("OS"); final String shorthand; } diff --git a/transformation/src/main/java/org/opensearch/migrations/Version.java b/transformation/src/main/java/org/opensearch/migrations/Version.java index ec11678b5..329a42e78 100644 --- a/transformation/src/main/java/org/opensearch/migrations/Version.java +++ b/transformation/src/main/java/org/opensearch/migrations/Version.java @@ -1,5 +1,7 @@ package org.opensearch.migrations; +import java.util.Arrays; + import lombok.Builder; import lombok.EqualsAndHashCode; import lombok.Getter; @@ -23,20 +25,20 @@ public static Version fromString(final String raw) throws RuntimeException { var builder = Version.builder(); var remainingString = raw.toLowerCase(); - for (var flavor : Flavor.values()) { - if (remainingString.startsWith(flavor.name().toLowerCase())) { - remainingString = remainingString.substring(flavor.name().length()); - builder.flavor(flavor); - break; - } else if (remainingString.startsWith(flavor.shorthand.toLowerCase())) { - remainingString = remainingString.substring(flavor.shorthand.length()); - builder.flavor(flavor); - break; - } - } + var finalRemainingString = remainingString; + var matchedFlavor = Arrays.stream(Flavor.values()) + .filter(flavor -> finalRemainingString.startsWith(flavor.name().toLowerCase()) || + finalRemainingString.startsWith(flavor.shorthand.toLowerCase())) + .findFirst(); - if (remainingString.equals(raw.toLowerCase())) { - throw new RuntimeException("Unable to determine build flavor from '" + raw +"'"); + if (matchedFlavor.isPresent()) { + Flavor flavor = matchedFlavor.get(); + remainingString = remainingString.startsWith(flavor.name().toLowerCase()) ? + remainingString.substring(flavor.name().length()) : + remainingString.substring(flavor.shorthand.length()); + builder.flavor(flavor); + } else { + throw new IllegalArgumentException("Unable to determine build flavor from '" + raw +"'"); } try { @@ -59,7 +61,7 @@ public static Version fromString(final String raw) throws RuntimeException { } return builder.build(); } catch (Exception e) { - throw new RuntimeException("Unable to parse version numbers from the string '" + raw + "'\r\n", e); + throw new IllegalArgumentException("Unable to parse version numbers from the string '" + raw + "'\r\n", e); } } } diff --git a/transformation/src/main/java/org/opensearch/migrations/VersionMatchers.java b/transformation/src/main/java/org/opensearch/migrations/VersionMatchers.java index 4f9efc7c8..6ebc77fc1 100644 --- a/transformation/src/main/java/org/opensearch/migrations/VersionMatchers.java +++ b/transformation/src/main/java/org/opensearch/migrations/VersionMatchers.java @@ -26,18 +26,14 @@ private static Predicate matchesMajorVersion(final Version version) { } private static Predicate matchesMinorVersion(final Version version) { - return other -> { - return matchesMajorVersion(version) - .and(other2 -> version.getMinor() == other2.getMinor()) - .test(other); - }; + return other -> matchesMajorVersion(version) + .and(other2 -> version.getMinor() == other2.getMinor()) + .test(other); } private static Predicate equalOrGreaterThanMinorVersion(final Version version) { - return other -> { - return matchesMajorVersion(version) - .and(other2 -> version.getMinor() <= other2.getMinor()) - .test(other); - }; + return other -> matchesMajorVersion(version) + .and(other2 -> version.getMinor() <= other2.getMinor()) + .test(other); } } diff --git a/transformation/src/main/java/org/opensearch/migrations/transformation/CanApplyResult.java b/transformation/src/main/java/org/opensearch/migrations/transformation/CanApplyResult.java index 8889046f1..a1cdac073 100644 --- a/transformation/src/main/java/org/opensearch/migrations/transformation/CanApplyResult.java +++ b/transformation/src/main/java/org/opensearch/migrations/transformation/CanApplyResult.java @@ -7,8 +7,9 @@ * The result after checking if a transformer can be applied to an entity */ public abstract class CanApplyResult { - public final static CanApplyResult YES = new Yes(); - public final static CanApplyResult NO = new No(); + private CanApplyResult() {} + public static final CanApplyResult YES = new Yes(); + public static final CanApplyResult NO = new No(); /** Yes, the transformation can be applied */ public static final class Yes extends CanApplyResult {} diff --git a/transformation/src/main/java/org/opensearch/migrations/transformation/entity/Entity.java b/transformation/src/main/java/org/opensearch/migrations/transformation/entity/Entity.java index 542d3e965..47161001d 100644 --- a/transformation/src/main/java/org/opensearch/migrations/transformation/entity/Entity.java +++ b/transformation/src/main/java/org/opensearch/migrations/transformation/entity/Entity.java @@ -15,5 +15,5 @@ public interface Entity { /** * Gets the underlying entity as an ObjectNode, supports read and write operations */ - ObjectNode rawJson(); + ObjectNode getRawJson(); } diff --git a/transformation/src/main/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemoval.java b/transformation/src/main/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemoval.java index 77aab8c67..5ada91268 100644 --- a/transformation/src/main/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemoval.java +++ b/transformation/src/main/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemoval.java @@ -11,7 +11,7 @@ /** * Supports transformation of the Index Mapping types that were changed from mutliple types to a single type between ES 6 to ES 7 - * + * * Example: * Starting state (ES 6): * { @@ -26,7 +26,7 @@ * } * ] * } - * + * * Ending state (ES 7): * { * "mappings": { @@ -39,30 +39,32 @@ */ public class IndexMappingTypeRemoval implements TransformationRule { + public static final String MAPPINGS_KEY = "mappings"; + @Override public CanApplyResult canApply(final Index index) { - final var mappingNode = index.rawJson().get("mappings"); + final var mappingNode = index.getRawJson().get(MAPPINGS_KEY); if (mappingNode == null) { return CanApplyResult.NO; } - // Detect unsupported multiple type mappings, eg: - // { "mappings": [{ "foo": {...}}, { "bar": {...} }] } - // { "mappings": [{ "foo": {...}, "bar": {...} }] } - if (mappingNode.isArray()) { - if (mappingNode.size() > 1 || mappingNode.get(0).size() > 1) { - return new Unsupported("Multiple mapping types are not supported"); - } + + // Detect unsupported multiple type mappings: + // 1.
{"mappings": [{ "foo": {...} }, { "bar": {...} }]}
+ // 2.
{"mappings": [{ "foo": {...}, "bar": {...}  }]}
+ if (mappingNode.isArray() && (mappingNode.size() > 1 || mappingNode.get(0).size() > 1)) { + return new Unsupported("Multiple mapping types are not supported"); } - // Detect if there is no intermediate type node - // { "mappings": { "_doc": { "properties": { } } } } + // Check for absence of intermediate type node + // 1.
{"mappings": {"properties": {...} }}
if (mappingNode.isObject() && mappingNode.get("properties") != null) { return CanApplyResult.NO; } - // There is a type under mappings, e.g. { "mappings": [{ "foo": {...} }] } + // There is a type under mappings + // 1.
{ "mappings": [{ "foo": {...} }] }
return CanApplyResult.YES; } @@ -72,7 +74,7 @@ public boolean applyTransformation(final Index index) { return false; } - final var mappingsNode = index.rawJson().get("mappings"); + final var mappingsNode = index.getRawJson().get(MAPPINGS_KEY); // Handle array case if (mappingsNode.isArray()) { final var mappingsInnerNode = (ObjectNode) mappingsNode.get(0); @@ -82,7 +84,7 @@ public boolean applyTransformation(final Index index) { mappingsInnerNode.remove(typeName); typeNode.fields().forEachRemaining(node -> mappingsInnerNode.set(node.getKey(), node.getValue())); - index.rawJson().set("mappings", mappingsInnerNode); + index.getRawJson().set(MAPPINGS_KEY, mappingsInnerNode); } if (mappingsNode.isObject()) { diff --git a/transformation/src/test/java/org/opensearch/migrations/VersionTest.java b/transformation/src/test/java/org/opensearch/migrations/VersionTest.java index 17b0150d7..62f936224 100644 --- a/transformation/src/test/java/org/opensearch/migrations/VersionTest.java +++ b/transformation/src/test/java/org/opensearch/migrations/VersionTest.java @@ -10,7 +10,7 @@ public class VersionTest { @Test void fromString() throws ParseException { - var expected = Version.builder().flavor(Flavor.OpenSearch).major(1).minor(3).patch(18).build(); + var expected = Version.builder().flavor(Flavor.OPENSEARCH).major(1).minor(3).patch(18).build(); assertThat(Version.fromString("OpenSearch 1.3.18"), equalTo(expected)); assertThat(Version.fromString("Opensearch 1.3.18"), equalTo(expected)); assertThat(Version.fromString("Opensearch 1.3.18"), equalTo(expected)); @@ -23,7 +23,7 @@ void fromString() throws ParseException { @Test void fromString_defaultPatch() throws ParseException { - var expected = Version.builder().flavor(Flavor.OpenSearch).major(1).minor(3).patch(0).build(); + var expected = Version.builder().flavor(Flavor.OPENSEARCH).major(1).minor(3).patch(0).build(); assertThat(Version.fromString("OpenSearch 1.3.0"), equalTo(expected)); assertThat(Version.fromString("OpenSearch 1.3.x"), equalTo(expected)); assertThat(Version.fromString("OpenSearch 1.3"), equalTo(expected)); @@ -31,7 +31,7 @@ void fromString_defaultPatch() throws ParseException { @Test void fromString_defaultMinor() throws ParseException { - var expected = Version.builder().flavor(Flavor.OpenSearch).major(1).minor(0).patch(0).build(); + var expected = Version.builder().flavor(Flavor.OPENSEARCH).major(1).minor(0).patch(0).build(); assertThat(Version.fromString("OpenSearch 1.0.0"), equalTo(expected)); assertThat(Version.fromString("OpenSearch 1.0"), equalTo(expected)); assertThat(Version.fromString("OpenSearch 1.x.x"), equalTo(expected)); diff --git a/transformation/src/test/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemovalTest.java b/transformation/src/test/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemovalTest.java index 1cda71c03..3d9e77068 100644 --- a/transformation/src/test/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemovalTest.java +++ b/transformation/src/test/java/org/opensearch/migrations/transformation/rules/IndexMappingTypeRemovalTest.java @@ -102,14 +102,14 @@ public ObjectNode indexSettingJson(final String mappingSection) { private CanApplyResult canApply(final ObjectNode indexJson) { var transformer = new IndexMappingTypeRemoval(); var index = mock(Index.class); - Mockito.when(index.rawJson()).thenReturn(indexJson); + Mockito.when(index.getRawJson()).thenReturn(indexJson); return transformer.canApply(index); } private boolean applyTransformation(final ObjectNode indexJson) { var transformer = new IndexMappingTypeRemoval(); var index = mock(Index.class); - Mockito.when(index.rawJson()).thenReturn(indexJson); + Mockito.when(index.getRawJson()).thenReturn(indexJson); log.atInfo().setMessage("Original\n{}").addArgument(indexJson.toPrettyString()).log(); var wasChanged = transformer.applyTransformation(index); diff --git a/transformation/src/test/resources/log4j2.properties b/transformation/src/test/resources/log4j2.properties index b698220a3..25b4245ff 100644 --- a/transformation/src/test/resources/log4j2.properties +++ b/transformation/src/test/resources/log4j2.properties @@ -1,4 +1,6 @@ -status = error +status = WARN + +property.ownedPackagesLogLevel=${sys:migrationLogLevel:-debug} # Root logger options rootLogger.level = debug @@ -9,3 +11,9 @@ appender.console.type = Console appender.console.name = Console appender.console.layout.type = PatternLayout appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS}{UTC} %p %c{1.} [%t] %m%equals{ ctx=%mdc}{ ctx=\{\}}{}%n + +# Allow customization of owned package logs +logger.rfs.name = com.rfs +logger.rfs.level = ${ownedPackagesLogLevel} +logger.migration.name = org.opensearch.migrations +logger.migration.level = ${ownedPackagesLogLevel}