From a1a159649517a1f85b5b9068b0ade4837f33159f Mon Sep 17 00:00:00 2001 From: Yupeng Fu Date: Wed, 29 Jan 2025 10:59:03 -0800 Subject: [PATCH] [Pull-based Ingestion] Introduce the new pull-based ingestion engine, APIs, and Kafka plugin (#16958) * add ingestion modules Signed-off-by: Yupeng Fu * stream poller wip Signed-off-by: Yupeng Fu * update ingestion engine Signed-off-by: Yupeng Fu * kafka container Signed-off-by: Yupeng Fu * more updates Signed-off-by: Yupeng Fu * local update Signed-off-by: Yupeng Fu * add batch_start/end to stream poller Signed-off-by: Yupeng Fu * add index settings Signed-off-by: Yupeng Fu * local change Signed-off-by: Yupeng Fu * pass docmapper Signed-off-by: Yupeng Fu * basic recovery Signed-off-by: Yupeng Fu * add kafka ingestion as plugin Signed-off-by: Yupeng Fu * add integration test for kafka plugin Signed-off-by: Yupeng Fu * cleanup Signed-off-by: Yupeng Fu * use byte[] for message payload type Signed-off-by: Yupeng Fu * javadocs Signed-off-by: Yupeng Fu * add ingestionEngineTest Signed-off-by: Yupeng Fu * test recovery test in ingestionEngineTest Signed-off-by: Yupeng Fu * unit tests for kafka plugin Signed-off-by: Yupeng Fu * style fix Signed-off-by: Yupeng Fu * add license Signed-off-by: Yupeng Fu * more unit tests Signed-off-by: Yupeng Fu * cleanup Signed-off-by: Yupeng Fu * use a blocking queue to pass polled messages to the processor for processing Signed-off-by: Yupeng Fu * address comments also remove security policy from bootstrap files Signed-off-by: Yupeng Fu * support _op_type in message processing Signed-off-by: Yupeng Fu * simplify ingestion source class Signed-off-by: Yupeng Fu * address more comments Signed-off-by: Yupeng Fu * kafka client sha Signed-off-by: Yupeng Fu * fix style Signed-off-by: Yupeng Fu * more style fix Signed-off-by: Yupeng Fu * add changelog Signed-off-by: Yupeng Fu * use latest gradle feature Signed-off-by: Yupeng Fu * try fix thread leak Signed-off-by: Yupeng Fu * comments Signed-off-by: Yupeng Fu * address comments Signed-off-by: Yupeng Fu * ingestionEngine creation failure tes Signed-off-by: Yupeng Fu * suppress removal warning Signed-off-by: Yupeng Fu * support lucene10 Signed-off-by: Yupeng Fu * address comments Signed-off-by: Yupeng Fu --------- Signed-off-by: Yupeng Fu --- CHANGELOG-3.0.md | 1 + gradle/missing-javadoc.gradle | 1 + plugins/ingestion-kafka/build.gradle | 120 ++ .../licenses/kafka-clients-3.8.1.jar.sha1 | 1 + .../licenses/kafka-clients-LICENSE.txt | 202 ++++ .../licenses/kafka-clients-NOTICE.txt | 19 + .../licenses/slf4j-api-1.7.36.jar.sha1 | 1 + .../licenses/slf4j-api-LICENSE.txt | 24 + .../licenses/slf4j-api-NOTICE.txt | 0 .../plugin/kafka/IngestFromKafkaIT.java | 131 +++ .../plugin/kafka/KafkaConsumerFactory.java | 45 + .../opensearch/plugin/kafka/KafkaMessage.java | 43 + .../opensearch/plugin/kafka/KafkaOffset.java | 97 ++ .../plugin/kafka/KafkaPartitionConsumer.java | 188 +++ .../opensearch/plugin/kafka/KafkaPlugin.java | 41 + .../plugin/kafka/KafkaSourceConfig.java | 48 + .../plugin-metadata/plugin-security.policy | 21 + .../kafka/KafkaConsumerFactoryTests.java | 39 + .../plugin/kafka/KafkaMessageTests.java | 33 + .../plugin/kafka/KafkaOffsetTests.java | 83 ++ .../kafka/KafkaPartitionConsumerTests.java | 125 ++ .../plugin/kafka/KafkaSourceConfigTests.java | 33 + .../opensearch/plugin/kafka/KafkaUtils.java | 87 ++ server/build.gradle | 2 +- .../cluster/metadata/IndexMetadata.java | 79 ++ .../cluster/metadata/IngestionSource.java | 62 + .../common/settings/IndexScopedSettings.java | 5 + .../org/opensearch/index/IndexService.java | 7 +- .../index/IngestionConsumerFactory.java | 49 + .../index/IngestionShardConsumer.java | 89 ++ .../index/IngestionShardPointer.java | 51 + .../java/org/opensearch/index/Message.java | 19 + .../opensearch/index/engine/EngineConfig.java | 13 + .../index/engine/EngineConfigFactory.java | 5 +- .../index/engine/IngestionEngine.java | 1015 +++++++++++++++++ .../opensearch/index/engine/NoOpEngine.java | 18 +- .../index/engine/ReadOnlyEngine.java | 17 +- .../opensearch/index/shard/IndexShard.java | 12 +- .../opensearch/index/translog/Translog.java | 15 + .../opensearch/indices/IndicesService.java | 28 + .../pollingingest/DefaultStreamPoller.java | 286 +++++ .../pollingingest/IngestionEngineFactory.java | 36 + .../MessageProcessorRunnable.java | 236 ++++ .../indices/pollingingest/StreamPoller.java | 71 ++ .../indices/pollingingest/package-info.java | 10 + .../main/java/org/opensearch/node/Node.java | 8 + .../plugins/IngestionConsumerPlugin.java | 36 + .../bootstrap/test-framework.policy | 1 + .../metadata/IngestionSourceTests.java | 67 ++ .../engine/EngineConfigFactoryTests.java | 2 + .../index/engine/FakeIngestionSource.java | 183 +++ .../index/engine/IngestionEngineTests.java | 189 +++ .../DefaultStreamPollerTests.java | 168 +++ .../pollingingest/MessageProcessorTests.java | 99 ++ .../snapshots/SnapshotResiliencyTests.java | 1 + .../index/engine/EngineTestCase.java | 41 +- .../test/OpenSearchIntegTestCase.java | 7 + 57 files changed, 4273 insertions(+), 37 deletions(-) create mode 100644 plugins/ingestion-kafka/build.gradle create mode 100644 plugins/ingestion-kafka/licenses/kafka-clients-3.8.1.jar.sha1 create mode 100644 plugins/ingestion-kafka/licenses/kafka-clients-LICENSE.txt create mode 100644 plugins/ingestion-kafka/licenses/kafka-clients-NOTICE.txt create mode 100644 plugins/ingestion-kafka/licenses/slf4j-api-1.7.36.jar.sha1 create mode 100644 plugins/ingestion-kafka/licenses/slf4j-api-LICENSE.txt create mode 100644 plugins/ingestion-kafka/licenses/slf4j-api-NOTICE.txt create mode 100644 plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/IngestFromKafkaIT.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaConsumerFactory.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaMessage.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaOffset.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPartitionConsumer.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPlugin.java create mode 100644 plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaSourceConfig.java create mode 100644 plugins/ingestion-kafka/src/main/plugin-metadata/plugin-security.policy create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaConsumerFactoryTests.java create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaMessageTests.java create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaOffsetTests.java create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaPartitionConsumerTests.java create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaSourceConfigTests.java create mode 100644 plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaUtils.java create mode 100644 server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java create mode 100644 server/src/main/java/org/opensearch/index/IngestionConsumerFactory.java create mode 100644 server/src/main/java/org/opensearch/index/IngestionShardConsumer.java create mode 100644 server/src/main/java/org/opensearch/index/IngestionShardPointer.java create mode 100644 server/src/main/java/org/opensearch/index/Message.java create mode 100644 server/src/main/java/org/opensearch/index/engine/IngestionEngine.java create mode 100644 server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java create mode 100644 server/src/main/java/org/opensearch/indices/pollingingest/IngestionEngineFactory.java create mode 100644 server/src/main/java/org/opensearch/indices/pollingingest/MessageProcessorRunnable.java create mode 100644 server/src/main/java/org/opensearch/indices/pollingingest/StreamPoller.java create mode 100644 server/src/main/java/org/opensearch/indices/pollingingest/package-info.java create mode 100644 server/src/main/java/org/opensearch/plugins/IngestionConsumerPlugin.java create mode 100644 server/src/test/java/org/opensearch/cluster/metadata/IngestionSourceTests.java create mode 100644 server/src/test/java/org/opensearch/index/engine/FakeIngestionSource.java create mode 100644 server/src/test/java/org/opensearch/index/engine/IngestionEngineTests.java create mode 100644 server/src/test/java/org/opensearch/indices/pollingingest/DefaultStreamPollerTests.java create mode 100644 server/src/test/java/org/opensearch/indices/pollingingest/MessageProcessorTests.java diff --git a/CHANGELOG-3.0.md b/CHANGELOG-3.0.md index 8d8adfd1e3566..bd7fd5fb621ce 100644 --- a/CHANGELOG-3.0.md +++ b/CHANGELOG-3.0.md @@ -13,6 +13,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - GHA to verify checklist items completion in PR descriptions ([#10800](https://github.com/opensearch-project/OpenSearch/pull/10800)) - Allow to pass the list settings through environment variables (like [], ["a", "b", "c"], ...) ([#10625](https://github.com/opensearch-project/OpenSearch/pull/10625)) - Views, simplify data access and manipulation by providing a virtual layer over one or more indices ([#11957](https://github.com/opensearch-project/OpenSearch/pull/11957)) +- Added pull-based Ingestion (APIs, for ingestion source, a Kafka plugin, and IngestionEngine that pulls data from the ingestion source) ([#16958](https://github.com/opensearch-project/OpenSearch/pull/16958)) ### Dependencies - Update Apache Lucene to 10.1.0 ([#16366](https://github.com/opensearch-project/OpenSearch/pull/16366)) diff --git a/gradle/missing-javadoc.gradle b/gradle/missing-javadoc.gradle index 342417b3b40ab..d65a318b096a9 100644 --- a/gradle/missing-javadoc.gradle +++ b/gradle/missing-javadoc.gradle @@ -140,6 +140,7 @@ configure([ project(":plugins:discovery-gce:qa:gce"), project(":plugins:identity-shiro"), project(":plugins:ingest-attachment"), + project(":plugins:ingestion-kafka"), project(":plugins:mapper-annotated-text"), project(":plugins:mapper-murmur3"), project(":plugins:mapper-size"), diff --git a/plugins/ingestion-kafka/build.gradle b/plugins/ingestion-kafka/build.gradle new file mode 100644 index 0000000000000..0d5344de213e3 --- /dev/null +++ b/plugins/ingestion-kafka/build.gradle @@ -0,0 +1,120 @@ +/* +* SPDX-License-Identifier: Apache-2.0 +* +* The OpenSearch Contributors require contributions made to +* this file be licensed under the Apache-2.0 license or a +* compatible open source license. +* +* Modifications Copyright OpenSearch Contributors. See +* GitHub history for details. +*/ + +apply plugin: 'opensearch.internal-cluster-test' + +opensearchplugin { + description = 'Pull-based ingestion plugin to consume from Kafka' + classname = 'org.opensearch.plugin.kafka.KafkaPlugin' +} + +versions << [ + 'kafka': '3.8.1', + 'docker': '3.3.6', + 'testcontainers': '1.19.7', + 'ducttape': '1.0.8', +] + +dependencies { + // kafka + api "org.slf4j:slf4j-api:${versions.slf4j}" + api "org.apache.kafka:kafka-clients:${versions.kafka}" + + // test + testImplementation "com.github.docker-java:docker-java-api:${versions.docker}" + testImplementation "com.github.docker-java:docker-java-transport:${versions.docker}" + testImplementation "com.github.docker-java:docker-java-transport-zerodep:${versions.docker}" + testImplementation "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" + testImplementation "org.testcontainers:testcontainers:${versions.testcontainers}" + testImplementation "org.testcontainers:kafka:${versions.testcontainers}" + testImplementation "org.rnorth.duct-tape:duct-tape:${versions.ducttape}" + testImplementation "org.apache.commons:commons-compress:${versions.commonscompress}" + testImplementation "commons-io:commons-io:${versions.commonsio}" + testImplementation 'org.awaitility:awaitility:4.2.0' +} + +internalClusterTest{ + environment 'TESTCONTAINERS_RYUK_DISABLED', 'true' + // TODO: Adding permission in plugin-security.policy doesn't seem to work. + systemProperty 'tests.security.manager', 'false' +} + +thirdPartyAudit { + ignoreMissingClasses( + 'com.fasterxml.jackson.databind.JsonNode', + 'com.fasterxml.jackson.databind.ObjectMapper', + 'com.fasterxml.jackson.databind.node.ArrayNode', + 'com.fasterxml.jackson.databind.node.BooleanNode', + 'com.fasterxml.jackson.databind.node.JsonNodeFactory', + 'com.fasterxml.jackson.databind.node.JsonNodeType', + 'com.fasterxml.jackson.databind.node.NullNode', + 'com.fasterxml.jackson.databind.node.ObjectNode', + 'net.jpountz.lz4.LZ4Compressor', + 'net.jpountz.lz4.LZ4Factory', + 'net.jpountz.lz4.LZ4SafeDecompressor', + 'net.jpountz.util.SafeUtils', + 'net.jpountz.xxhash.XXHash32', + 'net.jpountz.xxhash.XXHashFactory', + 'org.slf4j.impl.StaticLoggerBinder', + 'org.slf4j.impl.StaticMDCBinder', + 'org.slf4j.impl.StaticMarkerBinder', + 'com.google.common.util.concurrent.ListenableFuture', + 'io.grpc.BindableService', + 'io.grpc.CallOptions', + 'io.grpc.Channel', + 'io.grpc.MethodDescriptor', + 'io.grpc.MethodDescriptor$Builder', + 'io.grpc.MethodDescriptor$MethodType', + 'io.grpc.ServerServiceDefinition', + 'io.grpc.ServerServiceDefinition$Builder', + 'io.grpc.ServiceDescriptor', + 'io.grpc.ServiceDescriptor$Builder', + 'io.grpc.protobuf.ProtoFileDescriptorSupplier', + 'io.grpc.protobuf.ProtoMethodDescriptorSupplier', + 'io.grpc.protobuf.ProtoServiceDescriptorSupplier', + 'io.grpc.protobuf.ProtoUtils', + 'io.grpc.stub.AbstractAsyncStub', + 'io.grpc.stub.AbstractBlockingStub', + 'io.grpc.stub.AbstractFutureStub', + 'io.grpc.stub.AbstractStub$StubFactory', + 'io.grpc.stub.ClientCalls', + 'io.grpc.stub.ServerCalls', + 'io.grpc.stub.ServerCalls$BidiStreamingMethod', + 'io.grpc.stub.ServerCalls$ClientStreamingMethod', + 'io.grpc.stub.ServerCalls$ServerStreamingMethod', + 'io.grpc.stub.ServerCalls$UnaryMethod', + 'io.grpc.stub.StreamObserver', + 'org.jose4j.http.Get', + 'org.jose4j.jwa.AlgorithmConstraints', + 'org.jose4j.jwk.HttpsJwks', + 'org.jose4j.jwk.JsonWebKey', + 'org.jose4j.jwk.JsonWebKeySet', + 'org.jose4j.jwk.VerificationJwkSelector', + 'org.jose4j.jws.JsonWebSignature', + 'org.jose4j.jwt.JwtClaims', + 'org.jose4j.jwt.NumericDate', + 'org.jose4j.jwt.consumer.InvalidJwtException', + 'org.jose4j.jwt.consumer.JwtConsumer', + 'org.jose4j.jwt.consumer.JwtConsumerBuilder', + 'org.jose4j.jwt.consumer.JwtContext', + 'org.jose4j.jwx.Headers', + 'org.jose4j.keys.resolvers.VerificationKeyResolver', + ) + ignoreViolations( + 'org.apache.kafka.shaded.com.google.protobuf.MessageSchema', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil$1', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil$Android32MemoryAccessor', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil$Android64MemoryAccessor', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil$JvmMemoryAccessor', + 'org.apache.kafka.shaded.com.google.protobuf.UnsafeUtil$MemoryAccessor', + ) +} diff --git a/plugins/ingestion-kafka/licenses/kafka-clients-3.8.1.jar.sha1 b/plugins/ingestion-kafka/licenses/kafka-clients-3.8.1.jar.sha1 new file mode 100644 index 0000000000000..3864a4eb6a0dd --- /dev/null +++ b/plugins/ingestion-kafka/licenses/kafka-clients-3.8.1.jar.sha1 @@ -0,0 +1 @@ +fd79e3aa252c6d818334e9c0bac8166b426e498c \ No newline at end of file diff --git a/plugins/ingestion-kafka/licenses/kafka-clients-LICENSE.txt b/plugins/ingestion-kafka/licenses/kafka-clients-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/ingestion-kafka/licenses/kafka-clients-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/plugins/ingestion-kafka/licenses/kafka-clients-NOTICE.txt b/plugins/ingestion-kafka/licenses/kafka-clients-NOTICE.txt new file mode 100644 index 0000000000000..677b5251d9bfa --- /dev/null +++ b/plugins/ingestion-kafka/licenses/kafka-clients-NOTICE.txt @@ -0,0 +1,19 @@ +Apache Kafka +Copyright 2022 The Apache Software Foundation. + +This product includes software developed at +The Apache Software Foundation (https://www.apache.org/). + +This distribution has a binary dependency on jersey, which is available under the CDDL +License. The source code of jersey can be found at https://github.com/jersey/jersey/. + +The streams-scala (streams/streams-scala) module was donated by Lightbend and the original code was copyrighted by them: +Copyright (C) 2018 Lightbend Inc. +Copyright (C) 2017-2018 Alexis Seigneurin. + +This project contains the following code copied from Apache Hadoop: +clients/src/main/java/org/apache/kafka/common/utils/PureJavaCrc32C.java +Some portions of this file Copyright (c) 2004-2006 Intel Corporation and licensed under the BSD license. + +This project contains the following code copied from Apache Hive: +streams/src/main/java/org/apache/kafka/streams/state/internals/Murmur3.java diff --git a/plugins/ingestion-kafka/licenses/slf4j-api-1.7.36.jar.sha1 b/plugins/ingestion-kafka/licenses/slf4j-api-1.7.36.jar.sha1 new file mode 100644 index 0000000000000..77b9917528382 --- /dev/null +++ b/plugins/ingestion-kafka/licenses/slf4j-api-1.7.36.jar.sha1 @@ -0,0 +1 @@ +6c62681a2f655b49963a5983b8b0950a6120ae14 \ No newline at end of file diff --git a/plugins/ingestion-kafka/licenses/slf4j-api-LICENSE.txt b/plugins/ingestion-kafka/licenses/slf4j-api-LICENSE.txt new file mode 100644 index 0000000000000..1a3d053237bec --- /dev/null +++ b/plugins/ingestion-kafka/licenses/slf4j-api-LICENSE.txt @@ -0,0 +1,24 @@ +Copyright (c) 2004-2022 QOS.ch Sarl (Switzerland) +All rights reserved. + +Permission is hereby granted, free of charge, to any person obtaining +a copy of this software and associated documentation files (the +"Software"), to deal in the Software without restriction, including +without limitation the rights to use, copy, modify, merge, publish, +distribute, sublicense, and/or sell copies of the Software, and to +permit persons to whom the Software is furnished to do so, subject to +the following conditions: + +The above copyright notice and this permission notice shall be +included in all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND +NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE +LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION +OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION +WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. + + + diff --git a/plugins/ingestion-kafka/licenses/slf4j-api-NOTICE.txt b/plugins/ingestion-kafka/licenses/slf4j-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/IngestFromKafkaIT.java b/plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/IngestFromKafkaIT.java new file mode 100644 index 0000000000000..c726489d12626 --- /dev/null +++ b/plugins/ingestion-kafka/src/internalClusterTest/java/org/opensearch/plugin/kafka/IngestFromKafkaIT.java @@ -0,0 +1,131 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering; + +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.StringSerializer; +import org.opensearch.action.admin.cluster.node.info.NodeInfo; +import org.opensearch.action.admin.cluster.node.info.NodesInfoRequest; +import org.opensearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.opensearch.action.admin.cluster.node.info.PluginsAndModules; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.query.RangeQueryBuilder; +import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.PluginInfo; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.junit.Assert; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.utility.DockerImageName; + +import static org.hamcrest.Matchers.is; +import static org.awaitility.Awaitility.await; + +/** + * Integration test for Kafka ingestion + */ +@ThreadLeakLingering(linger = 15000) // wait for container pull thread to die +public class IngestFromKafkaIT extends OpenSearchIntegTestCase { + static final String topicName = "test"; + + private KafkaContainer kafka; + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(KafkaPlugin.class); + } + + /** + * test ingestion-kafka-plugin is installed + */ + public void testPluginsAreInstalled() { + NodesInfoRequest nodesInfoRequest = new NodesInfoRequest(); + nodesInfoRequest.addMetric(NodesInfoRequest.Metric.PLUGINS.metricName()); + NodesInfoResponse nodesInfoResponse = OpenSearchIntegTestCase.client().admin().cluster().nodesInfo(nodesInfoRequest).actionGet(); + List pluginInfos = nodesInfoResponse.getNodes() + .stream() + .flatMap( + (Function>) nodeInfo -> nodeInfo.getInfo(PluginsAndModules.class).getPluginInfos().stream() + ) + .collect(Collectors.toList()); + Assert.assertTrue( + pluginInfos.stream().anyMatch(pluginInfo -> pluginInfo.getName().equals("org.opensearch.plugin.kafka.KafkaPlugin")) + ); + } + + public void testKafkaIngestion() { + setupKafka(); + // create an index with ingestion source from kafka + createIndex( + "test", + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("ingestion_source.type", "kafka") + .put("ingestion_source.pointer.init.reset", "earliest") + .put("ingestion_source.param.topic", "test") + .put("ingestion_source.param.bootstrap_servers", kafka.getBootstrapServers()) + .build(), + "{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}" + ); + + RangeQueryBuilder query = new RangeQueryBuilder("age").gte(21); + await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { + refresh("test"); + SearchResponse response = client().prepareSearch("test").setQuery(query).get(); + assertThat(response.getHits().getTotalHits().value(), is(1L)); + }); + + stopKafka(); + } + + private void setupKafka() { + kafka = new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:6.2.1")) + // disable topic auto creation + .withEnv("KAFKA_AUTO_CREATE_TOPICS_ENABLE", "false"); + kafka.start(); + prepareKafkaData(); + } + + private void stopKafka() { + kafka.stop(); + } + + private void prepareKafkaData() { + String boostrapServers = kafka.getBootstrapServers(); + KafkaUtils.createTopic(topicName, 1, boostrapServers); + Properties props = new Properties(); + props.put("bootstrap.servers", kafka.getBootstrapServers()); + Producer producer = new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()); + producer.send(new ProducerRecord<>(topicName, "null", "{\"_id\":\"1\",\"_source\":{\"name\":\"bob\", \"age\": 24}}")); + producer.send( + new ProducerRecord<>( + topicName, + "null", + "{\"_id\":\"2\", \"_op_type:\":\"index\",\"_source\":{\"name\":\"alice\", \"age\": 20}}" + ) + ); + producer.close(); + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaConsumerFactory.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaConsumerFactory.java new file mode 100644 index 0000000000000..83c2480a3c985 --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaConsumerFactory.java @@ -0,0 +1,45 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.index.IngestionConsumerFactory; + +import java.util.Map; + +/** + * Factory for creating Kafka consumers + */ +public class KafkaConsumerFactory implements IngestionConsumerFactory { + + /** + * Configuration for the Kafka source + */ + protected KafkaSourceConfig config; + + /** + * Constructor. + */ + public KafkaConsumerFactory() {} + + @Override + public void initialize(Map params) { + config = new KafkaSourceConfig(params); + } + + @Override + public KafkaPartitionConsumer createShardConsumer(String clientId, int shardId) { + assert config != null; + return new KafkaPartitionConsumer(clientId, config, shardId); + } + + @Override + public KafkaOffset parsePointerFromString(String pointer) { + return new KafkaOffset(Long.valueOf(pointer)); + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaMessage.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaMessage.java new file mode 100644 index 0000000000000..7ff13f48ef846 --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaMessage.java @@ -0,0 +1,43 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.common.Nullable; +import org.opensearch.index.Message; + +/** + * Kafka message + */ +public class KafkaMessage implements Message { + private final byte[] key; + private final byte[] payload; + + /** + * Constructor + * @param key the key of the message + * @param payload the payload of the message + */ + public KafkaMessage(@Nullable byte[] key, byte[] payload) { + this.key = key; + this.payload = payload; + } + + /** + * Get the key of the message + * @return the key of the message + */ + public byte[] getKey() { + return key; + } + + @Override + public byte[] getPayload() { + return payload; + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaOffset.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaOffset.java new file mode 100644 index 0000000000000..0a1c11d5daa8b --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaOffset.java @@ -0,0 +1,97 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.search.Query; +import org.opensearch.index.IngestionShardPointer; + +import java.nio.ByteBuffer; + +/** + * Kafka offset. + */ +public class KafkaOffset implements IngestionShardPointer { + + private final long offset; + + /** + * Constructor + * @param offset the offset + */ + public KafkaOffset(long offset) { + assert offset >= 0; + this.offset = offset; + } + + @Override + public byte[] serialize() { + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); + buffer.putLong(offset); + return buffer.array(); + } + + @Override + public String asString() { + return String.valueOf(offset); + } + + @Override + public Field asPointField(String fieldName) { + return new LongPoint(fieldName, offset); + } + + @Override + public Query newRangeQueryGreaterThan(String fieldName) { + return LongPoint.newRangeQuery(fieldName, offset, Long.MAX_VALUE); + } + + /** + * Get the offset + * @return the offset + */ + public long getOffset() { + return offset; + } + + @Override + public String toString() { + return "KafkaOffset{" + "offset=" + offset + '}'; + } + + @Override + public int compareTo(IngestionShardPointer o) { + if (o == null) { + throw new IllegalArgumentException("the pointer is null"); + } + if (!(o instanceof KafkaOffset)) { + throw new IllegalArgumentException("the pointer is of type " + o.getClass() + " and not KafkaOffset"); + } + KafkaOffset other = (KafkaOffset) o; + return Long.compare(offset, other.offset); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + KafkaOffset that = (KafkaOffset) o; + return offset == that.offset; + } + + @Override + public int hashCode() { + return Long.hashCode(offset); + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPartitionConsumer.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPartitionConsumer.java new file mode 100644 index 0000000000000..a20e52a06eecd --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPartitionConsumer.java @@ -0,0 +1,188 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.index.IngestionShardPointer; + +import java.io.IOException; +import java.security.AccessController; +import java.security.PrivilegedAction; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.TimeoutException; + +/** + * Kafka consumer to read messages from a Kafka partition + */ +@SuppressWarnings("removal") +public class KafkaPartitionConsumer implements IngestionShardConsumer { + private static final Logger logger = LogManager.getLogger(KafkaPartitionConsumer.class); + + /** + * The Kafka consumer + */ + protected final Consumer consumer; + // TODO: make this configurable + private final int timeoutMillis = 1000; + + private long lastFetchedOffset = -1; + final String clientId; + final TopicPartition topicPartition; + + /** + * Constructor + * @param clientId the client id + * @param config the Kafka source config + * @param partitionId the partition id + */ + public KafkaPartitionConsumer(String clientId, KafkaSourceConfig config, int partitionId) { + this(clientId, config, partitionId, createConsumer(clientId, config)); + } + + /** + * Constructor, visible for testing + * @param clientId the client id + * @param config the Kafka source config + * @param partitionId the partition id + * @param consumer the created Kafka consumer + */ + protected KafkaPartitionConsumer(String clientId, KafkaSourceConfig config, int partitionId, Consumer consumer) { + this.clientId = clientId; + this.consumer = consumer; + String topic = config.getTopic(); + List partitionInfos = AccessController.doPrivileged( + (PrivilegedAction>) () -> consumer.partitionsFor(topic, Duration.ofMillis(timeoutMillis)) + ); + if (partitionInfos == null) { + throw new IllegalArgumentException("Topic " + topic + " does not exist"); + } + if (partitionId >= partitionInfos.size()) { + throw new IllegalArgumentException("Partition " + partitionId + " does not exist in topic " + topic); + } + topicPartition = new TopicPartition(topic, partitionId); + consumer.assign(Collections.singletonList(topicPartition)); + logger.info("Kafka consumer created for topic {} partition {}", topic, partitionId); + } + + /** + * Create a Kafka consumer. visible for testing + * @param clientId the client id + * @param config the Kafka source config + * @return the Kafka consumer + */ + protected static Consumer createConsumer(String clientId, KafkaSourceConfig config) { + Properties consumerProp = new Properties(); + consumerProp.put("bootstrap.servers", config.getBootstrapServers()); + consumerProp.put("client.id", clientId); + // TODO: why Class org.apache.kafka.common.serialization.StringDeserializer could not be found if set the deserializer as prop? + // consumerProp.put("key.deserializer", + // "org.apache.kafka.common.serialization.StringDeserializer"); + // consumerProp.put("value.deserializer", + // "org.apache.kafka.common.serialization.StringDeserializer"); + // + // wrap the kafka consumer creation in a privileged block to apply plugin security policies + return AccessController.doPrivileged( + (PrivilegedAction>) () -> new KafkaConsumer<>( + consumerProp, + new ByteArrayDeserializer(), + new ByteArrayDeserializer() + ) + ); + } + + @Override + public List> readNext(KafkaOffset offset, long maxMessages, int timeoutMillis) + throws TimeoutException { + List> records = AccessController.doPrivileged( + (PrivilegedAction>>) () -> fetch(offset.getOffset(), maxMessages, timeoutMillis) + ); + return records; + } + + @Override + public KafkaOffset nextPointer() { + return new KafkaOffset(lastFetchedOffset + 1); + } + + @Override + public IngestionShardPointer earliestPointer() { + long startOffset = AccessController.doPrivileged( + (PrivilegedAction) () -> consumer.beginningOffsets(Collections.singletonList(topicPartition)) + .getOrDefault(topicPartition, 0L) + ); + return new KafkaOffset(startOffset); + } + + @Override + public IngestionShardPointer latestPointer() { + long endOffset = AccessController.doPrivileged( + (PrivilegedAction) () -> consumer.endOffsets(Collections.singletonList(topicPartition)).getOrDefault(topicPartition, 0L) + ); + return new KafkaOffset(endOffset); + } + + private synchronized List> fetch(long startOffset, long maxMessages, int timeoutMillis) { + if (lastFetchedOffset < 0 || lastFetchedOffset != startOffset - 1) { + logger.info("Seeking to offset {}", startOffset); + consumer.seek(topicPartition, startOffset); + // update the last fetched offset so that we don't need to seek again if no more messages to fetch + lastFetchedOffset = startOffset - 1; + } + + ConsumerRecords consumerRecords = consumer.poll(Duration.ofMillis(timeoutMillis)); + List> messageAndOffsets = consumerRecords.records(topicPartition); + + long endOffset = startOffset + maxMessages; + List> results = new ArrayList<>(); + + for (ConsumerRecord messageAndOffset : messageAndOffsets) { + long currentOffset = messageAndOffset.offset(); + if (currentOffset >= endOffset) { + // fetched more message than max + break; + } + lastFetchedOffset = currentOffset; + KafkaOffset kafkaOffset = new KafkaOffset(currentOffset); + KafkaMessage message = new KafkaMessage(messageAndOffset.key(), messageAndOffset.value()); + results.add(new ReadResult<>(kafkaOffset, message)); + } + return results; + } + + @Override + public int getShardId() { + return topicPartition.partition(); + } + + @Override + public void close() throws IOException { + consumer.close(); + } + + /** + * Get the client id + * @return the client id + */ + public String getClientId() { + return clientId; + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPlugin.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPlugin.java new file mode 100644 index 0000000000000..8de2834c40583 --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaPlugin.java @@ -0,0 +1,41 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.index.IngestionConsumerFactory; +import org.opensearch.plugins.IngestionConsumerPlugin; +import org.opensearch.plugins.Plugin; + +import java.util.Map; + +/** + * A plugin for ingestion source of Kafka. + */ +public class KafkaPlugin extends Plugin implements IngestionConsumerPlugin { + /** + * The type of the ingestion source. + */ + public static final String TYPE = "KAFKA"; + + /** + * Constructor. + */ + public KafkaPlugin() {} + + @SuppressWarnings("rawtypes") + @Override + public Map getIngestionConsumerFactories() { + return Map.of(TYPE, new KafkaConsumerFactory()); + } + + @Override + public String getType() { + return TYPE; + } +} diff --git a/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaSourceConfig.java b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaSourceConfig.java new file mode 100644 index 0000000000000..099300c6e5767 --- /dev/null +++ b/plugins/ingestion-kafka/src/main/java/org/opensearch/plugin/kafka/KafkaSourceConfig.java @@ -0,0 +1,48 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import java.util.Map; +import java.util.Objects; + +/** + * Class encapsulating the configuration of a Kafka source. + */ +public class KafkaSourceConfig { + private final String topic; + private final String bootstrapServers; + + /** + * Constructor + * @param params the configuration parameters + */ + public KafkaSourceConfig(Map params) { + // TODO: better parsing and validation + this.topic = (String) Objects.requireNonNull(params.get("topic")); + this.bootstrapServers = (String) Objects.requireNonNull(params.get("bootstrap_servers")); + assert this.bootstrapServers != null; + } + + /** + * Get the topic name + * @return the topic name + */ + public String getTopic() { + return topic; + } + + /** + * Get the bootstrap servers + * + * @return the bootstrap servers + */ + public String getBootstrapServers() { + return bootstrapServers; + } +} diff --git a/plugins/ingestion-kafka/src/main/plugin-metadata/plugin-security.policy b/plugins/ingestion-kafka/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 0000000000000..2fec30325339d --- /dev/null +++ b/plugins/ingestion-kafka/src/main/plugin-metadata/plugin-security.policy @@ -0,0 +1,21 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +grant { + // needed because Kafka Consumer needs the access to MBeans + permission javax.management.MBeanServerPermission "createMBeanServer"; + permission javax.management.MBeanTrustPermission "register"; + permission javax.management.MBeanPermission "*", "*"; + permission java.lang.management.ManagementPermission "monitor"; + permission java.lang.management.ManagementPermission "control"; + + // Allow host/ip name service lookups + permission java.net.SocketPermission "*", "connect"; + permission java.net.SocketPermission "*", "resolve"; +}; + diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaConsumerFactoryTests.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaConsumerFactoryTests.java new file mode 100644 index 0000000000000..deaa4b1f0b369 --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaConsumerFactoryTests.java @@ -0,0 +1,39 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Assert; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaConsumerFactoryTests extends OpenSearchTestCase { + public void testInitialize() { + KafkaConsumerFactory factory = new KafkaConsumerFactory(); + Map params = new HashMap<>(); + params.put("topic", "test-topic"); + params.put("bootstrap_servers", "localhost:9092"); + + factory.initialize(params); + + KafkaSourceConfig config = factory.config; + Assert.assertNotNull("Config should be initialized", config); + Assert.assertEquals("Topic should be correctly initialized", "test-topic", config.getTopic()); + Assert.assertEquals("Bootstrap servers should be correctly initialized", "localhost:9092", config.getBootstrapServers()); + } + + public void testParsePointerFromString() { + KafkaConsumerFactory factory = new KafkaConsumerFactory(); + KafkaOffset offset = factory.parsePointerFromString("12345"); + + Assert.assertNotNull("Offset should be parsed", offset); + Assert.assertEquals("Offset value should be correctly parsed", 12345L, offset.getOffset()); + } +} diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaMessageTests.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaMessageTests.java new file mode 100644 index 0000000000000..a7601f7a1284e --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaMessageTests.java @@ -0,0 +1,33 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Assert; + +public class KafkaMessageTests extends OpenSearchTestCase { + public void testConstructorAndGetters() { + byte[] key = { 1, 2, 3 }; + byte[] payload = { 4, 5, 6 }; + + KafkaMessage message = new KafkaMessage(key, payload); + + Assert.assertArrayEquals(key, message.getKey()); + Assert.assertArrayEquals(payload, message.getPayload()); + } + + public void testConstructorWithNullKey() { + byte[] payload = { 4, 5, 6 }; + + KafkaMessage message = new KafkaMessage(null, payload); + + assertNull(message.getKey()); + Assert.assertArrayEquals(payload, message.getPayload()); + } +} diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaOffsetTests.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaOffsetTests.java new file mode 100644 index 0000000000000..c4170c5682073 --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaOffsetTests.java @@ -0,0 +1,83 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.search.PointRangeQuery; +import org.apache.lucene.search.Query; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Assert; + +import java.nio.ByteBuffer; + +public class KafkaOffsetTests extends OpenSearchTestCase { + + public void testConstructorAndGetters() { + long offset = 12345L; + KafkaOffset kafkaOffset = new KafkaOffset(offset); + + Assert.assertEquals("The offset should be correctly initialized and returned", offset, kafkaOffset.getOffset()); + } + + public void testEqualsAndHashCode() { + long offset1 = 12345L; + long offset2 = 67890L; + KafkaOffset kafkaOffset1 = new KafkaOffset(offset1); + KafkaOffset kafkaOffset2 = new KafkaOffset(offset1); + KafkaOffset kafkaOffset3 = new KafkaOffset(offset2); + + Assert.assertTrue("Offsets with the same value should be equal", kafkaOffset1.equals(kafkaOffset2)); + Assert.assertFalse("Offsets with different values should not be equal", kafkaOffset1.equals(kafkaOffset3)); + Assert.assertEquals("Hash codes for equal offsets should be the same", kafkaOffset1.hashCode(), kafkaOffset2.hashCode()); + Assert.assertNotEquals("Hash codes for different offsets should not be the same", kafkaOffset1.hashCode(), kafkaOffset3.hashCode()); + } + + public void testSerialize() { + long offset = 12345L; + KafkaOffset kafkaOffset = new KafkaOffset(offset); + byte[] expectedBytes = ByteBuffer.allocate(Long.BYTES).putLong(offset).array(); + + Assert.assertArrayEquals("The serialized bytes should be correct", expectedBytes, kafkaOffset.serialize()); + } + + public void testAsString() { + long offset = 12345L; + KafkaOffset kafkaOffset = new KafkaOffset(offset); + + Assert.assertEquals("The string representation should be correct", String.valueOf(offset), kafkaOffset.asString()); + } + + public void testAsPointField() { + long offset = 12345L; + KafkaOffset kafkaOffset = new KafkaOffset(offset); + Field field = kafkaOffset.asPointField("offsetField"); + + Assert.assertTrue("The field should be an instance of LongPoint", field instanceof LongPoint); + } + + public void testNewRangeQueryGreaterThan() { + long offset = 12345L; + KafkaOffset kafkaOffset = new KafkaOffset(offset); + Query query = kafkaOffset.newRangeQueryGreaterThan("offsetField"); + + Assert.assertTrue("The query should be an instance of range query", query instanceof PointRangeQuery); + } + + public void testCompareTo() { + long offset1 = 12345L; + long offset2 = 67890L; + KafkaOffset kafkaOffset1 = new KafkaOffset(offset1); + KafkaOffset kafkaOffset2 = new KafkaOffset(offset2); + + Assert.assertTrue("The comparison should be correct", kafkaOffset1.compareTo(kafkaOffset2) < 0); + Assert.assertTrue("The comparison should be correct", kafkaOffset2.compareTo(kafkaOffset1) > 0); + Assert.assertTrue("The comparison should be correct", kafkaOffset1.compareTo(kafkaOffset1) == 0); + } +} diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaPartitionConsumerTests.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaPartitionConsumerTests.java new file mode 100644 index 0000000000000..96f639366d887 --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaPartitionConsumerTests.java @@ -0,0 +1,125 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.time.Duration; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KafkaPartitionConsumerTests extends OpenSearchTestCase { + + private KafkaSourceConfig config; + private KafkaConsumer mockConsumer; + private KafkaPartitionConsumer consumer; + + @SuppressWarnings("unchecked") + @Before + public void setUp() throws Exception { + super.setUp(); + Map params = new HashMap<>(); + params.put("topic", "test-topic"); + params.put("bootstrap_servers", "localhost:9092"); + + config = new KafkaSourceConfig(params); + mockConsumer = mock(KafkaConsumer.class); + // Mock the partitionsFor method + PartitionInfo partitionInfo = new PartitionInfo("test-topic", 0, null, null, null); + when(mockConsumer.partitionsFor(eq("test-topic"), any(Duration.class))).thenReturn(Collections.singletonList(partitionInfo)); + consumer = new KafkaPartitionConsumer("client1", config, 0, mockConsumer); + } + + public void testReadNext() throws Exception { + TopicPartition topicPartition = new TopicPartition("test-topic", 0); + ConsumerRecord record = new ConsumerRecord<>("test-topic", 0, 0, null, "message".getBytes(StandardCharsets.UTF_8)); + ConsumerRecords records = new ConsumerRecords<>( + Collections.singletonMap(topicPartition, Collections.singletonList(record)) + ); + + when(mockConsumer.poll(any(Duration.class))).thenReturn(records); + + List> result = consumer.readNext(new KafkaOffset(0), 10, 1000); + + assertEquals(1, result.size()); + assertEquals("message", new String(result.get(0).getMessage().getPayload(), StandardCharsets.UTF_8)); + assertEquals(1, consumer.nextPointer().getOffset()); + assertEquals(0, consumer.getShardId()); + assertEquals("client1", consumer.getClientId()); + } + + public void testEarliestPointer() { + TopicPartition topicPartition = new TopicPartition("test-topic", 0); + when(mockConsumer.beginningOffsets(Collections.singletonList(topicPartition))).thenReturn( + Collections.singletonMap(topicPartition, 0L) + ); + + KafkaOffset offset = (KafkaOffset) consumer.earliestPointer(); + + assertEquals(0L, offset.getOffset()); + } + + public void testLatestPointer() { + TopicPartition topicPartition = new TopicPartition("test-topic", 0); + when(mockConsumer.endOffsets(Collections.singletonList(topicPartition))).thenReturn(Collections.singletonMap(topicPartition, 10L)); + + KafkaOffset offset = (KafkaOffset) consumer.latestPointer(); + + assertEquals(10L, offset.getOffset()); + } + + public void testTopicDoesNotExist() { + Map params = new HashMap<>(); + params.put("topic", "non-existent-topic"); + params.put("bootstrap_servers", "localhost:9092"); + var kafkaSourceConfig = new KafkaSourceConfig(params); + when(mockConsumer.partitionsFor(eq("non-existent-topic"), any(Duration.class))).thenReturn(null); + try { + new KafkaPartitionConsumer("client1", kafkaSourceConfig, 0, mockConsumer); + fail("Expected IllegalArgumentException"); + } catch (IllegalArgumentException e) { + assertEquals("Topic non-existent-topic does not exist", e.getMessage()); + } + } + + public void testPartitionDoesNotExist() { + PartitionInfo partitionInfo = new PartitionInfo("test-topic", 0, null, null, null); + when(mockConsumer.partitionsFor(eq("test-topic"), any(Duration.class))).thenReturn(Collections.singletonList(partitionInfo)); + try { + new KafkaPartitionConsumer("client1", config, 1, mockConsumer); + fail("Expected IllegalArgumentException"); + } catch (IllegalArgumentException e) { + assertEquals("Partition 1 does not exist in topic test-topic", e.getMessage()); + } + } + + public void testCreateConsumer() { + String clientId = "test-client"; + Consumer consumer = KafkaPartitionConsumer.createConsumer(clientId, config); + + assertNotNull(consumer); + assertEquals(KafkaConsumer.class, consumer.getClass()); + } +} diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaSourceConfigTests.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaSourceConfigTests.java new file mode 100644 index 0000000000000..aa4ddb94f23fc --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaSourceConfigTests.java @@ -0,0 +1,33 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Assert; + +import java.util.HashMap; +import java.util.Map; + +public class KafkaSourceConfigTests extends OpenSearchTestCase { + + public void testConstructorAndGetters() { + Map params = new HashMap<>(); + params.put("topic", "topic"); + params.put("bootstrap_servers", "bootstrap"); + + KafkaSourceConfig config = new KafkaSourceConfig(params); + + Assert.assertEquals("The topic should be correctly initialized and returned", "topic", config.getTopic()); + Assert.assertEquals( + "The bootstrap servers should be correctly initialized and returned", + "bootstrap", + config.getBootstrapServers() + ); + } +} diff --git a/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaUtils.java b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaUtils.java new file mode 100644 index 0000000000000..39af56ea04ed7 --- /dev/null +++ b/plugins/ingestion-kafka/src/test/java/org/opensearch/plugin/kafka/KafkaUtils.java @@ -0,0 +1,87 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugin.kafka; + +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.KafkaAdminClient; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; + +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; + +import static org.awaitility.Awaitility.await; + +public class KafkaUtils { + private static final Logger LOGGER = LogManager.getLogger(KafkaUtils.class); + + /** + * Creates kafka topic + * + * @param topicName the topic name + * @param bootstrapServer kafka bootstrap server list + */ + public static void createTopic(String topicName, String bootstrapServer) { + createTopic(topicName, 1, bootstrapServer); + } + + public static void createTopic(String topicName, int numOfPartitions, String bootstrapServers) { + try { + getAdminClient(bootstrapServers, (client -> { + NewTopic newTopic = new NewTopic(topicName, numOfPartitions, (short) 1); + client.createTopics(List.of(newTopic)); + return true; + })); + + } catch (TopicExistsException e) { + // Catch TopicExistsException otherwise it will break maven-surefire-plugin + LOGGER.warn("Topic {} already existed", topicName); + } + + // validates topic is created + await().atMost(3, TimeUnit.SECONDS).until(() -> checkTopicExistence(topicName, bootstrapServers)); + } + + public static boolean checkTopicExistence(String topicName, String bootstrapServers) { + return getAdminClient(bootstrapServers, (client -> { + Map> topics = client.describeTopics(List.of(topicName)).values(); + + try { + return topics.containsKey(topicName) && topics.get(topicName).get().name().equals(topicName); + } catch (InterruptedException e) { + LOGGER.error("error on checkTopicExistence", e); + return false; + } catch (ExecutionException e) { + LOGGER.error("error on checkTopicExistence", e); + return false; + } + })); + } + + private static Rep getAdminClient(String bootstrapServer, Function function) { + AdminClient adminClient = KafkaAdminClient.create( + ImmutableMap.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServer, AdminClientConfig.CLIENT_ID_CONFIG, "test") + ); + try { + return function.apply(adminClient); + } finally { + adminClient.close(); + } + } +} diff --git a/server/build.gradle b/server/build.gradle index 82eafb07a7ad3..e7541cfdccdff 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -113,7 +113,7 @@ dependencies { // https://mvnrepository.com/artifact/org.roaringbitmap/RoaringBitmap api libs.roaringbitmap - + testImplementation 'org.awaitility:awaitility:4.2.0' testImplementation(project(":test:framework")) { // tests use the locally compiled version of server exclude group: 'org.opensearch', module: 'server' diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index e09630d813ebf..4e605dce498ab 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -71,6 +71,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.indices.pollingingest.StreamPoller; import org.opensearch.indices.replication.SegmentReplicationSource; import org.opensearch.indices.replication.common.ReplicationType; @@ -656,6 +657,63 @@ public static APIBlock readFrom(StreamInput input) throws IOException { Property.IndexScope ); + /** + * Used to specify the type for the ingestion index. If not specified, the ingestion source not enabled + */ + public static final String SETTING_INGESTION_SOURCE_TYPE = "index.ingestion_source.type"; + public static final String NONE_INGESTION_SOURCE_TYPE = "none"; + public static final Setting INGESTION_SOURCE_TYPE_SETTING = Setting.simpleString( + SETTING_INGESTION_SOURCE_TYPE, + NONE_INGESTION_SOURCE_TYPE, + new Setting.Validator<>() { + + @Override + public void validate(final String value) { + // TODO: validate this with the registered types in the ingestion source plugin + } + + @Override + public void validate(final String value, final Map, Object> settings) { + // TODO: validate this with the ingestion source params + } + }, + Property.IndexScope + ); + + /** + * Used to specify initial reset policy for the ingestion pointer. If not specified, default to the latest + */ + public static final String SETTING_INGESTION_SOURCE_POINTER_INIT_RESET = "index.ingestion_source.pointer.init.reset"; + public static final Setting INGESTION_SOURCE_POINTER_INIT_RESET_SETTING = Setting.simpleString( + SETTING_INGESTION_SOURCE_POINTER_INIT_RESET, + StreamPoller.ResetState.LATEST.name(), + new Setting.Validator<>() { + + @Override + public void validate(final String value) { + if (!(value.equalsIgnoreCase(StreamPoller.ResetState.LATEST.name()) + || value.equalsIgnoreCase(StreamPoller.ResetState.EARLIEST.name()))) { + throw new IllegalArgumentException( + "Invalid value for " + SETTING_INGESTION_SOURCE_POINTER_INIT_RESET + " [" + value + "]" + ); + } + } + + @Override + public void validate(final String value, final Map, Object> settings) {} + }, + Property.IndexScope, + Property.Dynamic + ); + + public static final Setting.AffixSetting INGESTION_SOURCE_PARAMS_SETTING = Setting.prefixKeySetting( + "index.ingestion_source.param.", + key -> new Setting<>(key, "", (value) -> { + // TODO: add ingestion source params validation + return value; + }, Property.IndexScope) + ); + /** * an internal index format description, allowing us to find out if this index is upgraded or needs upgrading */ @@ -683,6 +741,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { public static final String REMOTE_STORE_CUSTOM_KEY = "remote_store"; public static final String TRANSLOG_METADATA_KEY = "translog_metadata"; public static final String CONTEXT_KEY = "context"; + public static final String INGESTION_SOURCE_KEY = "ingestion_source"; public static final String INDEX_STATE_FILE_PREFIX = "state-"; @@ -863,6 +922,25 @@ public Version getCreationVersion() { return indexCreatedVersion; } + /** + * Gets the ingestion source. + * @return ingestion source, or null if ingestion source is not enabled + */ + public IngestionSource getIngestionSource() { + final String ingestionSourceType = INGESTION_SOURCE_TYPE_SETTING.get(settings); + if (ingestionSourceType != null && !(NONE_INGESTION_SOURCE_TYPE.equals(ingestionSourceType))) { + final String pointerInitReset = INGESTION_SOURCE_POINTER_INIT_RESET_SETTING.get(settings); + final Map ingestionSourceParams = INGESTION_SOURCE_PARAMS_SETTING.getAsMap(settings); + return new IngestionSource(ingestionSourceType, pointerInitReset, ingestionSourceParams); + } + return null; + } + + public boolean useIngestionSource() { + final String ingestionSourceType = INGESTION_SOURCE_TYPE_SETTING.get(settings); + return ingestionSourceType != null && !(NONE_INGESTION_SOURCE_TYPE.equals(ingestionSourceType)); + } + /** * Return the {@link Version} on which this index has been upgraded. This * information is typically useful for backward compatibility. @@ -1209,6 +1287,7 @@ public IndexMetadata apply(IndexMetadata part) { builder.rolloverInfos.putAll(rolloverInfos.apply(part.rolloverInfos)); builder.system(isSystem); builder.context(context); + // TODO: support ingestion source return builder.build(); } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java b/server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java new file mode 100644 index 0000000000000..583114d9ecbd2 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/metadata/IngestionSource.java @@ -0,0 +1,62 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.metadata; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.Map; +import java.util.Objects; + +/** + * Class encapsulating the configuration of an ingestion source. + */ +@ExperimentalApi +public class IngestionSource { + private String type; + private String pointerInitReset; + private Map params; + + public IngestionSource(String type, String pointerInitReset, Map params) { + this.type = type; + this.pointerInitReset = pointerInitReset; + this.params = params; + } + + public String getType() { + return type; + } + + public String getPointerInitReset() { + return pointerInitReset; + } + + public Map params() { + return params; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + IngestionSource ingestionSource = (IngestionSource) o; + return Objects.equals(type, ingestionSource.type) + && Objects.equals(pointerInitReset, ingestionSource.pointerInitReset) + && Objects.equals(params, ingestionSource.params); + } + + @Override + public int hashCode() { + return Objects.hash(type, pointerInitReset, params); + } + + @Override + public String toString() { + return "IngestionSource{" + "type='" + type + '\'' + ",pointer_init_reset='" + pointerInitReset + '\'' + ", params=" + params + '}'; + } +} diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 387ed0ed92680..b8ace8495ad96 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -257,6 +257,11 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.INDEX_CONTEXT_CREATED_VERSION, IndexSettings.INDEX_CONTEXT_CURRENT_VERSION, + // Settings for ingestion source + IndexMetadata.INGESTION_SOURCE_TYPE_SETTING, + IndexMetadata.INGESTION_SOURCE_POINTER_INIT_RESET_SETTING, + IndexMetadata.INGESTION_SOURCE_PARAMS_SETTING, + // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { Map groups = s.getAsGroups(); diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index f1b36194bf62d..c623b0c63866c 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -308,8 +308,11 @@ public IndexService( // kick off async ops for the first shard in this index this.refreshTask = new AsyncRefreshTask(this); this.trimTranslogTask = new AsyncTrimTranslogTask(this); - this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); - this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); + // disable these checks for ingestion source engine + if (!indexSettings.getIndexMetadata().useIngestionSource()) { + this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); + this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); + } if (READER_WRITER_SPLIT_EXPERIMENTAL_SETTING.get(indexSettings.getNodeSettings())) { this.asyncReplicationTask = new AsyncReplicationTask(this); } diff --git a/server/src/main/java/org/opensearch/index/IngestionConsumerFactory.java b/server/src/main/java/org/opensearch/index/IngestionConsumerFactory.java new file mode 100644 index 0000000000000..dec8596981024 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/IngestionConsumerFactory.java @@ -0,0 +1,49 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.util.Map; + +/** + * A factory for creating {@link IngestionShardConsumer}. + * + * @param the type of the {@link IngestionShardConsumer} + * @param

the type of the {@link IngestionShardPointer} + */ +@ExperimentalApi +public interface IngestionConsumerFactory { + /** + * Initialize the factory with the configuration parameters. This method is called once when the factory is created, + * and the parameters are parsed from the {@link org.opensearch.cluster.metadata.IngestionSource} in + * {@link org.opensearch.cluster.metadata.IndexMetadata}. + * @param params the configuration parameters to initialize the factory + */ + void initialize(Map params); + + /** + * Create a consumer to ingest messages from a shard of the streams. When the ingestion engine created per shard, + * this method is called to create the consumer in the poller. Before the invocation of this method, the configuration + * is passed to the factory through the {@link #initialize(Map)} method. + * + * @param clientId the client id assigned to the consumer + * @param shardId the id of the shard + * @return the created consumer + */ + T createShardConsumer(String clientId, int shardId); + + /** + * Parses the pointer from a string representation to the pointer object. This is used for recovering from the index + * checkpoints. + * @param pointer the string representation of the pointer + * @return the recovered pointer + */ + P parsePointerFromString(String pointer); +} diff --git a/server/src/main/java/org/opensearch/index/IngestionShardConsumer.java b/server/src/main/java/org/opensearch/index/IngestionShardConsumer.java new file mode 100644 index 0000000000000..02a9f5a18ebb1 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/IngestionShardConsumer.java @@ -0,0 +1,89 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index; + +import org.opensearch.common.annotation.ExperimentalApi; + +import java.io.Closeable; +import java.util.List; + +/** + * A consumer for reading messages from an ingestion shard. + * @param the type of the pointer to the message + * @param the type of the message + */ +@ExperimentalApi +public interface IngestionShardConsumer extends Closeable { + + /** + * A read result containing the pointer and the message + * @param the type of the pointer to the message + * @param the type of the message + */ + @ExperimentalApi + class ReadResult { + T pointer; + M message; + + /** + * Create a new read result + * @param pointer the pointer to the message + * @param message the message + */ + public ReadResult(T pointer, M message) { + this.pointer = pointer; + this.message = message; + } + + /** + * @return the pointer to the message + */ + public T getPointer() { + return pointer; + } + + /** + * @return the message + */ + public M getMessage() { + return message; + } + } + + /** + * Read the next set of messages from the source + * @param pointer the pointer to start reading from, inclusive + * @param maxMessages, the maximum number of messages to read, or -1 for no limit + * @param timeoutMillis the maximum time to wait for messages + * @throws java.util.concurrent.TimeoutException If the operation could not be completed within {@code timeoutMillis} + * milliseconds + * @return a list of messages read from the source + */ + List> readNext(T pointer, long maxMessages, int timeoutMillis) throws java.util.concurrent.TimeoutException; + + /** + * @return the next pointer to read from + */ + T nextPointer(); + + /** + * @return the earliest pointer in the shard + */ + IngestionShardPointer earliestPointer(); + + /** + * @return the latest pointer in the shard. The pointer points to the next offset of the last message in the stream. + */ + IngestionShardPointer latestPointer(); + + /** + * @return the shard id + */ + int getShardId(); +} diff --git a/server/src/main/java/org/opensearch/index/IngestionShardPointer.java b/server/src/main/java/org/opensearch/index/IngestionShardPointer.java new file mode 100644 index 0000000000000..fc9c14f1ca0f0 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/IngestionShardPointer.java @@ -0,0 +1,51 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index; + +import org.apache.lucene.document.Field; +import org.apache.lucene.search.Query; +import org.opensearch.common.annotation.ExperimentalApi; + +/** + * An interface for the pointer to a shard in the ingestion engine, and it is used to track the message offset of + * ingestion. + */ +@ExperimentalApi +public interface IngestionShardPointer extends Comparable { + String OFFSET_FIELD = "_offset"; + + /** + * Serialize the pointer to a byte array. + * @return the serialized byte array + */ + byte[] serialize(); + + /** + * Convert the pointer to a string. + * @return the string representation of the pointer + */ + String asString(); + + /** + * Creates a point field for this pointer. This is used to store the pointer in the index for range search during + * checkpoint recovery. + * @param fieldName the field name to create the point field + * @return the point field + */ + Field asPointField(String fieldName); + + /** + * Create a new range query for values greater than the pointer. This is used in recovering from the ingestion + * checkpoints. + * + * @param fieldName the field name to create the range query + * @return query for values greater than the pointer + */ + Query newRangeQueryGreaterThan(String fieldName); +} diff --git a/server/src/main/java/org/opensearch/index/Message.java b/server/src/main/java/org/opensearch/index/Message.java new file mode 100644 index 0000000000000..d2ac92c2cc2b5 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/Message.java @@ -0,0 +1,19 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index; + +import org.opensearch.common.annotation.ExperimentalApi; + +/** + * A message ingested from the ingestion source that contains an index operation + */ +@ExperimentalApi +public interface Message { + T getPayload(); +} diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index fe2ce470cda03..6540e69e7dfcd 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -53,6 +53,7 @@ import org.opensearch.index.codec.CodecAliases; import org.opensearch.index.codec.CodecService; import org.opensearch.index.codec.CodecSettings; +import org.opensearch.index.mapper.DocumentMapperForType; import org.opensearch.index.mapper.ParsedDocument; import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.index.store.Store; @@ -110,6 +111,7 @@ public final class EngineConfig { private final boolean isReadOnlyReplica; private final BooleanSupplier startedPrimarySupplier; private final Comparator leafSorter; + private final Supplier documentMapperForTypeSupplier; /** * A supplier of the outstanding retention leases. This is used during merged operations to determine which operations that have been @@ -296,6 +298,7 @@ private EngineConfig(Builder builder) { this.startedPrimarySupplier = builder.startedPrimarySupplier; this.translogFactory = builder.translogFactory; this.leafSorter = builder.leafSorter; + this.documentMapperForTypeSupplier = builder.documentMapperForTypeSupplier; } /** @@ -544,6 +547,10 @@ public TombstoneDocSupplier getTombstoneDocSupplier() { return tombstoneDocSupplier; } + public Supplier getDocumentMapperForTypeSupplier() { + return documentMapperForTypeSupplier; + } + public TranslogDeletionPolicyFactory getCustomTranslogDeletionPolicyFactory() { return translogDeletionPolicyFactory; } @@ -589,6 +596,7 @@ public static class Builder { private boolean isReadOnlyReplica; private BooleanSupplier startedPrimarySupplier; private TranslogFactory translogFactory = new InternalTranslogFactory(); + private Supplier documentMapperForTypeSupplier; Comparator leafSorter; public Builder shardId(ShardId shardId) { @@ -701,6 +709,11 @@ public Builder tombstoneDocSupplier(TombstoneDocSupplier tombstoneDocSupplier) { return this; } + public Builder documentMapperForTypeSupplier(Supplier documentMapperForTypeSupplier) { + this.documentMapperForTypeSupplier = documentMapperForTypeSupplier; + return this; + } + public Builder translogDeletionPolicyFactory(TranslogDeletionPolicyFactory translogDeletionPolicyFactory) { this.translogDeletionPolicyFactory = translogDeletionPolicyFactory; return this; diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java index 77e2f1c55201d..d892d6e95346c 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java @@ -25,6 +25,7 @@ import org.opensearch.index.codec.CodecService; import org.opensearch.index.codec.CodecServiceConfig; import org.opensearch.index.codec.CodecServiceFactory; +import org.opensearch.index.mapper.DocumentMapperForType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.index.store.Store; @@ -154,7 +155,8 @@ public EngineConfig newEngineConfig( boolean isReadOnlyReplica, BooleanSupplier startedPrimarySupplier, TranslogFactory translogFactory, - Comparator leafSorter + Comparator leafSorter, + Supplier documentMapperForTypeSupplier ) { CodecService codecServiceToUse = codecService; if (codecService == null && this.codecServiceFactory != null) { @@ -188,6 +190,7 @@ public EngineConfig newEngineConfig( .startedPrimarySupplier(startedPrimarySupplier) .translogFactory(translogFactory) .leafSorter(leafSorter) + .documentMapperForTypeSupplier(documentMapperForTypeSupplier) .build(); } diff --git a/server/src/main/java/org/opensearch/index/engine/IngestionEngine.java b/server/src/main/java/org/opensearch/index/engine/IngestionEngine.java new file mode 100644 index 0000000000000..2a1206d0a6ef2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/engine/IngestionEngine.java @@ -0,0 +1,1015 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.engine; + +import org.apache.lucene.analysis.standard.StandardAnalyzer; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.MergePolicy; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.index.ShuffleForcedMergePolicy; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.ReferenceManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.util.InfoStream; +import org.opensearch.ExceptionsHelper; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.metadata.IngestionSource; +import org.opensearch.common.Booleans; +import org.opensearch.common.Nullable; +import org.opensearch.common.SuppressForbidden; +import org.opensearch.common.concurrent.GatedCloseable; +import org.opensearch.common.lucene.LoggerInfoStream; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.concurrent.AbstractRunnable; +import org.opensearch.common.util.concurrent.ReleasableLock; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.unit.ByteSizeValue; +import org.opensearch.core.index.shard.ShardId; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.IngestionConsumerFactory; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.mapper.DocumentMapperForType; +import org.opensearch.index.mapper.IdFieldMapper; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.index.merge.MergeStats; +import org.opensearch.index.merge.OnGoingMerge; +import org.opensearch.index.seqno.SeqNoStats; +import org.opensearch.index.shard.OpenSearchMergePolicy; +import org.opensearch.index.translog.NoOpTranslogManager; +import org.opensearch.index.translog.Translog; +import org.opensearch.index.translog.TranslogCorruptedException; +import org.opensearch.index.translog.TranslogManager; +import org.opensearch.index.translog.TranslogStats; +import org.opensearch.indices.pollingingest.DefaultStreamPoller; +import org.opensearch.indices.pollingingest.StreamPoller; +import org.opensearch.search.suggest.completion.CompletionStats; +import org.opensearch.threadpool.ThreadPool; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.BiConsumer; +import java.util.function.BiFunction; +import java.util.function.UnaryOperator; + +import static org.opensearch.index.translog.Translog.EMPTY_TRANSLOG_SNAPSHOT; + +/** + * IngestionEngine is an engine that ingests data from a stream source. + */ +public class IngestionEngine extends Engine { + + private volatile SegmentInfos lastCommittedSegmentInfos; + private final CompletionStatsCache completionStatsCache; + private final IndexWriter indexWriter; + private final OpenSearchReaderManager internalReaderManager; + private final ExternalReaderManager externalReaderManager; + private final Lock flushLock = new ReentrantLock(); + private final ReentrantLock optimizeLock = new ReentrantLock(); + private final OpenSearchConcurrentMergeScheduler mergeScheduler; + private final AtomicBoolean shouldPeriodicallyFlushAfterBigMerge = new AtomicBoolean(false); + private final TranslogManager translogManager; + private final DocumentMapperForType documentMapperForType; + private final IngestionConsumerFactory ingestionConsumerFactory; + private StreamPoller streamPoller; + + /** + * UUID value that is updated every time the engine is force merged. + */ + @Nullable + private volatile String forceMergeUUID; + + public IngestionEngine(EngineConfig engineConfig, IngestionConsumerFactory ingestionConsumerFactory) { + super(engineConfig); + store.incRef(); + boolean success = false; + try { + this.lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); + this.completionStatsCache = new CompletionStatsCache(() -> acquireSearcher("completion_stats")); + IndexMetadata indexMetadata = engineConfig.getIndexSettings().getIndexMetadata(); + assert indexMetadata != null; + mergeScheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings()); + indexWriter = createWriter(); + externalReaderManager = createReaderManager(new InternalEngine.RefreshWarmerListener(logger, isClosed, engineConfig)); + internalReaderManager = externalReaderManager.internalReaderManager; + translogManager = new NoOpTranslogManager( + shardId, + readLock, + this::ensureOpen, + new TranslogStats(0, 0, 0, 0, 0), + EMPTY_TRANSLOG_SNAPSHOT + ); + documentMapperForType = engineConfig.getDocumentMapperForTypeSupplier().get(); + this.ingestionConsumerFactory = Objects.requireNonNull(ingestionConsumerFactory); + + success = true; + } catch (IOException | TranslogCorruptedException e) { + throw new EngineCreationFailureException(shardId, "failed to create engine", e); + } finally { + if (!success) { + if (streamPoller != null) { + try { + streamPoller.close(); + } catch (IOException e) { + logger.error("failed to close stream poller", e); + throw new RuntimeException(e); + } + } + if (!isClosed.get()) { + // failure, we need to dec the store reference + store.decRef(); + } + } + } + } + + /** + * Starts the ingestion engine to pull. + */ + public void start() { + IndexMetadata indexMetadata = engineConfig.getIndexSettings().getIndexMetadata(); + assert indexMetadata != null; + IngestionSource ingestionSource = Objects.requireNonNull(indexMetadata.getIngestionSource()); + + // initialize the ingestion consumer factory + this.ingestionConsumerFactory.initialize(ingestionSource.params()); + String clientId = engineConfig.getIndexSettings().getNodeName() + + "-" + + engineConfig.getIndexSettings().getIndex().getName() + + "-" + + engineConfig.getShardId().getId(); + IngestionShardConsumer ingestionShardConsumer = this.ingestionConsumerFactory.createShardConsumer( + clientId, + engineConfig.getShardId().getId() + ); + logger.info("created ingestion consumer for shard [{}]", engineConfig.getShardId()); + + Map commitData = commitDataAsMap(); + StreamPoller.ResetState resetState = StreamPoller.ResetState.valueOf( + ingestionSource.getPointerInitReset().toUpperCase(Locale.ROOT) + ); + IngestionShardPointer startPointer = null; + Set persistedPointers = new HashSet<>(); + if (commitData.containsKey(StreamPoller.BATCH_START)) { + // try recovering from commit data + String batchStartStr = commitData.get(StreamPoller.BATCH_START); + startPointer = this.ingestionConsumerFactory.parsePointerFromString(batchStartStr); + try (Searcher searcher = acquireSearcher("restore_offset", SearcherScope.INTERNAL)) { + persistedPointers = fetchPersistedOffsets(Lucene.wrapAllDocsLive(searcher.getDirectoryReader()), startPointer); + logger.info("recovered persisted pointers: {}", persistedPointers); + } catch (IOException e) { + throw new EngineCreationFailureException(config().getShardId(), "failed to restore offset", e); + } + // reset to none so the poller will poll from the startPointer + resetState = StreamPoller.ResetState.NONE; + } + + streamPoller = new DefaultStreamPoller(startPointer, persistedPointers, ingestionShardConsumer, this, resetState); + streamPoller.start(); + } + + private IndexWriter createWriter() throws IOException { + try { + final IndexWriterConfig iwc = getIndexWriterConfig(); + return createWriter(store.directory(), iwc); + } catch (LockObtainFailedException ex) { + logger.warn("could not lock IndexWriter", ex); + throw ex; + } + } + + public DocumentMapperForType getDocumentMapperForType() { + return documentMapperForType; + } + + protected Set fetchPersistedOffsets(DirectoryReader directoryReader, IngestionShardPointer batchStart) + throws IOException { + final IndexSearcher searcher = new IndexSearcher(directoryReader); + searcher.setQueryCache(null); + var query = batchStart.newRangeQueryGreaterThan(IngestionShardPointer.OFFSET_FIELD); + + // Execute the search + var topDocs = searcher.search(query, Integer.MAX_VALUE); + Set result = new HashSet<>(); + var storedFields = searcher.getIndexReader().storedFields(); + for (var scoreDoc : topDocs.scoreDocs) { + var doc = storedFields.document(scoreDoc.doc); + String valueStr = doc.get(IngestionShardPointer.OFFSET_FIELD); + IngestionShardPointer value = ingestionConsumerFactory.parsePointerFromString(valueStr); + result.add(value); + } + + refresh("restore_offset", SearcherScope.INTERNAL, true); + return result; + } + + /** + * a copy of ExternalReaderManager from InternalEngine + */ + @SuppressForbidden(reason = "reference counting is required here") + static final class ExternalReaderManager extends ReferenceManager { + private final BiConsumer refreshListener; + private final OpenSearchReaderManager internalReaderManager; + private boolean isWarmedUp; // guarded by refreshLock + + ExternalReaderManager( + OpenSearchReaderManager internalReaderManager, + BiConsumer refreshListener + ) throws IOException { + this.refreshListener = refreshListener; + this.internalReaderManager = internalReaderManager; + this.current = internalReaderManager.acquire(); // steal the reference without warming up + } + + @Override + protected OpenSearchDirectoryReader refreshIfNeeded(OpenSearchDirectoryReader referenceToRefresh) throws IOException { + // we simply run a blocking refresh on the internal reference manager and then steal it's reader + // it's a save operation since we acquire the reader which incs it's reference but then down the road + // steal it by calling incRef on the "stolen" reader + internalReaderManager.maybeRefreshBlocking(); + final OpenSearchDirectoryReader newReader = internalReaderManager.acquire(); + if (isWarmedUp == false || newReader != referenceToRefresh) { + boolean success = false; + try { + refreshListener.accept(newReader, isWarmedUp ? referenceToRefresh : null); + isWarmedUp = true; + success = true; + } finally { + if (success == false) { + internalReaderManager.release(newReader); + } + } + } + // nothing has changed - both ref managers share the same instance so we can use reference equality + if (referenceToRefresh == newReader) { + internalReaderManager.release(newReader); + return null; + } else { + return newReader; // steal the reference + } + } + + @Override + protected boolean tryIncRef(OpenSearchDirectoryReader reference) { + return reference.tryIncRef(); + } + + @Override + protected int getRefCount(OpenSearchDirectoryReader reference) { + return reference.getRefCount(); + } + + @Override + protected void decRef(OpenSearchDirectoryReader reference) throws IOException { + reference.decRef(); + } + } + + private ExternalReaderManager createReaderManager(InternalEngine.RefreshWarmerListener externalRefreshListener) throws EngineException { + boolean success = false; + OpenSearchReaderManager internalReaderManager = null; + try { + try { + final OpenSearchDirectoryReader directoryReader = OpenSearchDirectoryReader.wrap( + DirectoryReader.open(indexWriter), + shardId + ); + internalReaderManager = new OpenSearchReaderManager(directoryReader); + lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); + ExternalReaderManager externalReaderManager = new ExternalReaderManager(internalReaderManager, externalRefreshListener); + success = true; + return externalReaderManager; + } catch (IOException e) { + maybeFailEngine("start", e); + try { + indexWriter.rollback(); + } catch (IOException inner) { // iw is closed below + e.addSuppressed(inner); + } + throw new EngineCreationFailureException(shardId, "failed to open reader on writer", e); + } + } finally { + if (success == false) { // release everything we created on a failure + IOUtils.closeWhileHandlingException(internalReaderManager, indexWriter); + } + } + } + + // pkg-private for testing + IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { + return new IndexWriter(directory, iwc); + } + + private IndexWriterConfig getIndexWriterConfig() { + final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); + iwc.setCommitOnClose(false); // we by default don't commit on close + iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND); + // with tests.verbose, lucene sets this up: plumb to align with filesystem stream + boolean verbose = false; + try { + verbose = Boolean.parseBoolean(System.getProperty("tests.verbose")); + } catch (Exception ignore) {} + iwc.setInfoStream(verbose ? InfoStream.getDefault() : new LoggerInfoStream(logger)); + iwc.setMergeScheduler(mergeScheduler); + // set merge scheduler + MergePolicy mergePolicy = config().getMergePolicy(); + boolean shuffleForcedMerge = Booleans.parseBoolean(System.getProperty("opensearch.shuffle_forced_merge", Boolean.TRUE.toString())); + if (shuffleForcedMerge) { + // We wrap the merge policy for all indices even though it is mostly useful for time-based indices + // but there should be no overhead for other type of indices so it's simpler than adding a setting + // to enable it. + mergePolicy = new ShuffleForcedMergePolicy(mergePolicy); + } + + if (config().getIndexSettings().isMergeOnFlushEnabled()) { + final long maxFullFlushMergeWaitMillis = config().getIndexSettings().getMaxFullFlushMergeWaitTime().millis(); + if (maxFullFlushMergeWaitMillis > 0) { + iwc.setMaxFullFlushMergeWaitMillis(maxFullFlushMergeWaitMillis); + final Optional> mergeOnFlushPolicy = config().getIndexSettings().getMergeOnFlushPolicy(); + if (mergeOnFlushPolicy.isPresent()) { + mergePolicy = mergeOnFlushPolicy.get().apply(mergePolicy); + } + } + } else { + // Disable merge on refresh + iwc.setMaxFullFlushMergeWaitMillis(0); + } + + iwc.setCheckPendingFlushUpdate(config().getIndexSettings().isCheckPendingFlushEnabled()); + iwc.setMergePolicy(new OpenSearchMergePolicy(mergePolicy)); + iwc.setSimilarity(engineConfig.getSimilarity()); + iwc.setRAMBufferSizeMB(engineConfig.getIndexingBufferSize().getMbFrac()); + iwc.setCodec(engineConfig.getCodec()); + iwc.setUseCompoundFile(engineConfig.useCompoundFile()); + if (config().getIndexSort() != null) { + iwc.setIndexSort(config().getIndexSort()); + } + if (config().getLeafSorter() != null) { + iwc.setLeafSorter(config().getLeafSorter()); // The default segment search order + } + + return new IndexWriterConfig(new StandardAnalyzer()); + } + + @Override + public TranslogManager translogManager() { + // ingestion engine does not have translog + return translogManager; + } + + @Override + protected SegmentInfos getLastCommittedSegmentInfos() { + return lastCommittedSegmentInfos; + } + + @Override + protected SegmentInfos getLatestSegmentInfos() { + throw new UnsupportedOperationException(); + } + + @Override + public String getHistoryUUID() { + return loadHistoryUUID(lastCommittedSegmentInfos.userData); + } + + @Override + public long getWritingBytes() { + return 0; + } + + @Override + public CompletionStats completionStats(String... fieldNamePatterns) { + return completionStatsCache.get(fieldNamePatterns); + } + + @Override + public long getIndexThrottleTimeInMillis() { + return 0; + } + + @Override + public boolean isThrottled() { + return false; + } + + @Override + public IndexResult index(Index index) throws IOException { + assert Objects.equals(index.uid().field(), IdFieldMapper.NAME) : index.uid().field(); + ensureOpen(); + final IndexResult indexResult; + indexResult = indexIntoLucene(index); + return indexResult; + } + + private IndexResult indexIntoLucene(Index index) throws IOException { + // todo: handle updates + addDocs(index.docs(), indexWriter); + return new IndexResult(index.version(), index.primaryTerm(), index.seqNo(), true); + } + + private void addDocs(final List docs, final IndexWriter indexWriter) throws IOException { + if (docs.size() > 1) { + indexWriter.addDocuments(docs); + } else { + indexWriter.addDocument(docs.get(0)); + } + } + + @Override + public DeleteResult delete(Delete delete) throws IOException { + return null; + } + + @Override + public NoOpResult noOp(NoOp noOp) throws IOException { + ensureOpen(); + NoOpResult noOpResult = new NoOpResult(noOp.primaryTerm(), noOp.seqNo()); + return noOpResult; + } + + @Override + public GetResult get(Get get, BiFunction searcherFactory) throws EngineException { + return getFromSearcher(get, searcherFactory, SearcherScope.EXTERNAL); + } + + @Override + protected ReferenceManager getReferenceManager(SearcherScope scope) { + return externalReaderManager; + } + + @Override + public Closeable acquireHistoryRetentionLock() { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public Translog.Snapshot newChangesSnapshot( + String source, + long fromSeqNo, + long toSeqNo, + boolean requiredFullRange, + boolean accurateCount + ) throws IOException { + throw new UnsupportedOperationException("Not implemented"); + } + + @Override + public int countNumberOfHistoryOperations(String source, long fromSeqNo, long toSeqNumber) throws IOException { + return 0; + } + + @Override + public boolean hasCompleteOperationHistory(String reason, long startingSeqNo) { + return false; + } + + @Override + public long getMinRetainedSeqNo() { + return 0; + } + + @Override + public long getPersistedLocalCheckpoint() { + return 0; + } + + @Override + public long getProcessedLocalCheckpoint() { + return 0; + } + + @Override + public SeqNoStats getSeqNoStats(long globalCheckpoint) { + return null; + } + + @Override + public long getLastSyncedGlobalCheckpoint() { + return 0; + } + + @Override + public long getIndexBufferRAMBytesUsed() { + return 0; + } + + @Override + public List segments(boolean verbose) { + try (ReleasableLock lock = readLock.acquire()) { + Segment[] segmentsArr = getSegmentInfo(lastCommittedSegmentInfos, verbose); + + // fill in the merges flag + Set onGoingMerges = mergeScheduler.onGoingMerges(); + for (OnGoingMerge onGoingMerge : onGoingMerges) { + for (SegmentCommitInfo segmentInfoPerCommit : onGoingMerge.getMergedSegments()) { + for (Segment segment : segmentsArr) { + if (segment.getName().equals(segmentInfoPerCommit.info.name)) { + segment.mergeId = onGoingMerge.getId(); + break; + } + } + } + } + return Arrays.asList(segmentsArr); + } + } + + @Override + public void refresh(String source) throws EngineException { + refresh(source, SearcherScope.EXTERNAL, true); + } + + final boolean refresh(String source, SearcherScope scope, boolean block) throws EngineException { + boolean refreshed; + try { + // refresh does not need to hold readLock as ReferenceManager can handle correctly if the engine is closed in mid-way. + if (store.tryIncRef()) { + // increment the ref just to ensure nobody closes the store during a refresh + try { + // even though we maintain 2 managers we really do the heavy-lifting only once. + // the second refresh will only do the extra work we have to do for warming caches etc. + ReferenceManager referenceManager = getReferenceManager(scope); + // it is intentional that we never refresh both internal / external together + if (block) { + referenceManager.maybeRefreshBlocking(); + refreshed = true; + } else { + refreshed = referenceManager.maybeRefresh(); + } + } finally { + store.decRef(); + } + } else { + refreshed = false; + } + } catch (AlreadyClosedException e) { + failOnTragicEvent(e); + throw e; + } catch (Exception e) { + try { + failEngine("refresh failed source[" + source + "]", e); + } catch (Exception inner) { + e.addSuppressed(inner); + } + throw new RefreshFailedEngineException(shardId, e); + } + // We check for pruning in each delete request, but we also prune here e.g. in case a delete burst comes in and then no more deletes + // for a long time: + maybePruneDeletes(); + // TODO: use OS merge scheduler + mergeScheduler.refreshConfig(); + return refreshed; + } + + @Override + public boolean maybeRefresh(String source) throws EngineException { + return refresh(source, SearcherScope.EXTERNAL, false); + } + + @Override + public void writeIndexingBuffer() throws EngineException { + refresh("write indexing buffer", SearcherScope.INTERNAL, false); + } + + @Override + public boolean shouldPeriodicallyFlush() { + return false; + } + + @Override + public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + ensureOpen(); + if (force && waitIfOngoing == false) { + assert false : "wait_if_ongoing must be true for a force flush: force=" + force + " wait_if_ongoing=" + waitIfOngoing; + throw new IllegalArgumentException( + "wait_if_ongoing must be true for a force flush: force=" + force + " wait_if_ongoing=" + waitIfOngoing + ); + } + try (ReleasableLock lock = readLock.acquire()) { + ensureOpen(); + if (flushLock.tryLock() == false) { + // if we can't get the lock right away we block if needed otherwise barf + if (waitIfOngoing == false) { + return; + } + logger.trace("waiting for in-flight flush to finish"); + flushLock.lock(); + logger.trace("acquired flush lock after blocking"); + } else { + logger.trace("acquired flush lock immediately"); + } + try { + // Only flush if (1) Lucene has uncommitted docs, or (2) forced by caller, + // + // do we need to consider #3 and #4 as in InternalEngine? + // (3) the newly created commit points to a different translog generation (can free translog), + // or (4) the local checkpoint information in the last commit is stale, which slows down future recoveries. + boolean hasUncommittedChanges = indexWriter.hasUncommittedChanges(); + if (hasUncommittedChanges || force) { + logger.trace("starting commit for flush;"); + + // TODO: do we need to close the latest commit as done in InternalEngine? + commitIndexWriter(indexWriter); + + logger.trace("finished commit for flush"); + + // a temporary debugging to investigate test failure - issue#32827. Remove when the issue is resolved + logger.debug("new commit on flush, hasUncommittedChanges:{}, force:{}", hasUncommittedChanges, force); + + // we need to refresh in order to clear older version values + refresh("version_table_flush", SearcherScope.INTERNAL, true); + } + } catch (FlushFailedEngineException ex) { + maybeFailEngine("flush", ex); + throw ex; + } catch (IOException e) { + throw new FlushFailedEngineException(shardId, e); + } finally { + flushLock.unlock(); + } + } + } + + /** + * Commits the specified index writer. + * + * @param writer the index writer to commit + */ + protected void commitIndexWriter(final IndexWriter writer) throws IOException { + try { + writer.setLiveCommitData(() -> { + /* + * The user data captured the min and max range of the stream poller + */ + final Map commitData = new HashMap<>(2); + + commitData.put(StreamPoller.BATCH_START, streamPoller.getBatchStartPointer().asString()); + final String currentForceMergeUUID = forceMergeUUID; + if (currentForceMergeUUID != null) { + commitData.put(FORCE_MERGE_UUID_KEY, currentForceMergeUUID); + } + logger.trace("committing writer with commit data [{}]", commitData); + return commitData.entrySet().iterator(); + }); + writer.commit(); + } catch (final Exception ex) { + try { + failEngine("lucene commit failed", ex); + } catch (final Exception inner) { + ex.addSuppressed(inner); + } + throw ex; + } catch (final AssertionError e) { + /* + * If assertions are enabled, IndexWriter throws AssertionError on commit if any files don't exist, but tests that randomly + * throw FileNotFoundException or NoSuchFileException can also hit this. + */ + if (ExceptionsHelper.stackTrace(e).contains("org.apache.lucene.index.IndexWriter.filesExist")) { + final EngineException engineException = new EngineException(shardId, "failed to commit engine", e); + try { + failEngine("lucene commit failed", engineException); + } catch (final Exception inner) { + engineException.addSuppressed(inner); + } + throw engineException; + } else { + throw e; + } + } + } + + @Override + public MergeStats getMergeStats() { + return mergeScheduler.stats(); + } + + @Override + public void onSettingsChanged(TimeValue translogRetentionAge, ByteSizeValue translogRetentionSize, long softDeletesRetentionOps) { + mergeScheduler.refreshConfig(); + // TODO: do we need more? + } + + protected Map commitDataAsMap() { + return commitDataAsMap(indexWriter); + } + + /** + * Gets the commit data from {@link IndexWriter} as a map. + */ + protected static Map commitDataAsMap(final IndexWriter indexWriter) { + final Map commitData = new HashMap<>(8); + for (Map.Entry entry : indexWriter.getLiveCommitData()) { + commitData.put(entry.getKey(), entry.getValue()); + } + return commitData; + } + + @Override + public void forceMerge( + boolean flush, + int maxNumSegments, + boolean onlyExpungeDeletes, + boolean upgrade, + boolean upgradeOnlyAncientSegments, + String forceMergeUUID + ) throws EngineException, IOException { + /* + * We do NOT acquire the readlock here since we are waiting on the merges to finish + * that's fine since the IW.rollback should stop all the threads and trigger an IOException + * causing us to fail the forceMerge + * + * The way we implement upgrades is a bit hackish in the sense that we set an instance + * variable and that this setting will thus apply to the next forced merge that will be run. + * This is ok because (1) this is the only place we call forceMerge, (2) we have a single + * thread for optimize, and the 'optimizeLock' guarding this code, and (3) ConcurrentMergeScheduler + * syncs calls to findForcedMerges. + */ + assert indexWriter.getConfig().getMergePolicy() instanceof OpenSearchMergePolicy : "MergePolicy is " + + indexWriter.getConfig().getMergePolicy().getClass().getName(); + OpenSearchMergePolicy mp = (OpenSearchMergePolicy) indexWriter.getConfig().getMergePolicy(); + optimizeLock.lock(); + try { + ensureOpen(); + if (upgrade) { + logger.info("starting segment upgrade upgradeOnlyAncientSegments={}", upgradeOnlyAncientSegments); + mp.setUpgradeInProgress(true, upgradeOnlyAncientSegments); + } + store.incRef(); // increment the ref just to ensure nobody closes the store while we optimize + try { + if (onlyExpungeDeletes) { + assert upgrade == false; + indexWriter.forceMergeDeletes(true /* blocks and waits for merges*/); + } else if (maxNumSegments <= 0) { + assert upgrade == false; + indexWriter.maybeMerge(); + } else { + indexWriter.forceMerge(maxNumSegments, true /* blocks and waits for merges*/); + this.forceMergeUUID = forceMergeUUID; + } + if (flush) { + flush(false, true); + } + if (upgrade) { + logger.info("finished segment upgrade"); + } + } finally { + store.decRef(); + } + } catch (AlreadyClosedException ex) { + /* in this case we first check if the engine is still open. If so this exception is just fine + * and expected. We don't hold any locks while we block on forceMerge otherwise it would block + * closing the engine as well. If we are not closed we pass it on to failOnTragicEvent which ensures + * we are handling a tragic even exception here */ + ensureOpen(ex); + failOnTragicEvent(ex); + throw ex; + } catch (Exception e) { + try { + maybeFailEngine(FORCE_MERGE, e); + } catch (Exception inner) { + e.addSuppressed(inner); + } + throw e; + } finally { + try { + // reset it just to make sure we reset it in a case of an error + mp.setUpgradeInProgress(false, false); + } finally { + optimizeLock.unlock(); + } + } + } + + @Override + public GatedCloseable acquireLastIndexCommit(boolean flushFirst) throws EngineException { + store.incRef(); + try { + var reader = getReferenceManager(SearcherScope.INTERNAL).acquire(); + return new GatedCloseable<>(reader.getIndexCommit(), () -> { + store.decRef(); + getReferenceManager(SearcherScope.INTERNAL).release(reader); + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public GatedCloseable acquireSafeIndexCommit() throws EngineException { + // TODO: do we need this? likely not + return acquireLastIndexCommit(false); + } + + @Override + public SafeCommitInfo getSafeCommitInfo() { + // TODO: do we need this? + return SafeCommitInfo.EMPTY; + } + + @Override + protected void closeNoLock(String reason, CountDownLatch closedLatch) { + if (isClosed.compareAndSet(false, true)) { + assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread() + : "Either the write lock must be held or the engine must be currently be failing itself"; + try { + try { + IOUtils.close(externalReaderManager, internalReaderManager); + } catch (Exception e) { + logger.warn("Failed to close ReaderManager", e); + } + + // no need to commit in this case!, we snapshot before we close the shard, so translog and all sync'ed + logger.trace("rollback indexWriter"); + try { + indexWriter.rollback(); + } catch (AlreadyClosedException ex) { + failOnTragicEvent(ex); + throw ex; + } + logger.trace("rollback indexWriter done"); + } catch (Exception e) { + logger.warn("failed to rollback writer on close", e); + } finally { + try { + store.decRef(); + logger.debug("engine closed [{}]", reason); + } finally { + closedLatch.countDown(); + } + } + } + } + + private boolean failOnTragicEvent(AlreadyClosedException ex) { + final boolean engineFailed; + // if we are already closed due to some tragic exception + // we need to fail the engine. it might have already been failed before + // but we are double-checking it's failed and closed + if (indexWriter.isOpen() == false && indexWriter.getTragicException() != null) { + final Exception tragicException; + if (indexWriter.getTragicException() instanceof Exception) { + tragicException = (Exception) indexWriter.getTragicException(); + } else { + tragicException = new RuntimeException(indexWriter.getTragicException()); + } + failEngine("already closed by tragic event on the index writer", tragicException); + engineFailed = true; + } else if (failedEngine.get() == null && isClosed.get() == false) { // we are closed but the engine is not failed yet? + // this smells like a bug - we only expect ACE if we are in a fatal case ie. either translog or IW is closed by + // a tragic event or has closed itself. if that is not the case we are in a buggy state and raise an assertion error + throw new AssertionError("Unexpected AlreadyClosedException", ex); + } else { + engineFailed = false; + } + return engineFailed; + } + + private final class EngineMergeScheduler extends OpenSearchConcurrentMergeScheduler { + private final AtomicInteger numMergesInFlight = new AtomicInteger(0); + private final AtomicBoolean isThrottling = new AtomicBoolean(); + + EngineMergeScheduler(ShardId shardId, IndexSettings indexSettings) { + super(shardId, indexSettings); + } + + @Override + public synchronized void beforeMerge(OnGoingMerge merge) { + int maxNumMerges = mergeScheduler.getMaxMergeCount(); + if (numMergesInFlight.incrementAndGet() > maxNumMerges) { + if (isThrottling.getAndSet(true) == false) { + logger.info("now throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges); + activateThrottling(); + } + } + } + + @Override + public synchronized void afterMerge(OnGoingMerge merge) { + int maxNumMerges = mergeScheduler.getMaxMergeCount(); + if (numMergesInFlight.decrementAndGet() < maxNumMerges) { + if (isThrottling.getAndSet(false)) { + logger.info("stop throttling indexing: numMergesInFlight={}, maxNumMerges={}", numMergesInFlight, maxNumMerges); + deactivateThrottling(); + } + } + if (indexWriter.hasPendingMerges() == false + && System.nanoTime() - lastWriteNanos >= engineConfig.getFlushMergesAfter().nanos()) { + // NEVER do this on a merge thread since we acquire some locks blocking here and if we concurrently rollback the writer + // we deadlock on engine#close for instance. + engineConfig.getThreadPool().executor(ThreadPool.Names.FLUSH).execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (isClosed.get() == false) { + logger.warn("failed to flush after merge has finished"); + } + } + + @Override + protected void doRun() { + // if we have no pending merges and we are supposed to flush once merges have finished to + // free up transient disk usage of the (presumably biggish) segments that were just merged + flush(); + } + }); + } else if (merge.getTotalBytesSize() >= engineConfig.getIndexSettings().getFlushAfterMergeThresholdSize().getBytes()) { + // we hit a significant merge which would allow us to free up memory if we'd commit it hence on the next change + // we should execute a flush on the next operation if that's a flush after inactive or indexing a document. + // we could fork a thread and do it right away but we try to minimize forking and piggyback on outside events. + shouldPeriodicallyFlushAfterBigMerge.set(true); + } + } + + @Override + protected void handleMergeException(final Throwable exc) { + engineConfig.getThreadPool().generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + logger.debug("merge failure action rejected", e); + } + + @Override + protected void doRun() throws Exception { + /* + * We do this on another thread rather than the merge thread that we are initially called on so that we have complete + * confidence that the call stack does not contain catch statements that would cause the error that might be thrown + * here from being caught and never reaching the uncaught exception handler. + */ + failEngine(MERGE_FAILED, new MergePolicy.MergeException(exc)); + } + }); + } + } + + @Override + public void activateThrottling() { + // TODO: add this when we have a thread pool for indexing in parallel + } + + @Override + public void deactivateThrottling() { + // TODO: is this needed? + } + + @Override + public int fillSeqNoGaps(long primaryTerm) throws IOException { + // TODO: is this needed? + return 0; + } + + @Override + public void maybePruneDeletes() { + // no need to prune deletes in ingestion engine + } + + @Override + public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) { + // TODO: is this needed? + } + + @Override + public long getMaxSeqNoOfUpdatesOrDeletes() { + // TODO: is this needed? + return 0; + } + + @Override + public void advanceMaxSeqNoOfUpdatesOrDeletes(long maxSeqNoOfUpdatesOnPrimary) { + // TODO: is this needed? + } + + @Override + public void close() throws IOException { + if (streamPoller != null) { + streamPoller.close(); + } + super.close(); + } +} diff --git a/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java b/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java index 9071b0e7a1eb3..ac8e123e49204 100644 --- a/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NoOpEngine.java @@ -58,6 +58,8 @@ import java.util.Map; import java.util.function.Function; +import static org.opensearch.index.translog.Translog.EMPTY_TRANSLOG_SNAPSHOT; + /** * NoOpEngine is an engine implementation that does nothing but the bare minimum * required in order to have an engine. All attempts to do something (search, @@ -158,21 +160,7 @@ public DocsStats docStats() { */ public TranslogManager translogManager() { try { - return new NoOpTranslogManager(shardId, readLock, this::ensureOpen, this.translogStats, new Translog.Snapshot() { - @Override - public void close() {} - - @Override - public int totalOperations() { - return 0; - } - - @Override - public Translog.Operation next() { - return null; - } - - }) { + return new NoOpTranslogManager(shardId, readLock, this::ensureOpen, this.translogStats, EMPTY_TRANSLOG_SNAPSHOT) { /** * This implementation will trim existing translog files using a {@link TranslogDeletionPolicy} * that retains nothing but the last translog generation from safe commit. diff --git a/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java b/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java index 7ff3145055df8..1852f2fa92b74 100644 --- a/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/ReadOnlyEngine.java @@ -66,6 +66,8 @@ import java.util.function.BiFunction; import java.util.function.Function; +import static org.opensearch.index.translog.Translog.EMPTY_TRANSLOG_SNAPSHOT; + /** * A basic read-only engine that allows switching a shard to be true read-only temporarily or permanently. * Note: this engine can be opened side-by-side with a read-write engine but will not reflect any changes made to the read-write @@ -150,20 +152,7 @@ public ReadOnlyEngine( completionStatsCache = new CompletionStatsCache(() -> acquireSearcher("completion_stats")); - translogManager = new NoOpTranslogManager(shardId, readLock, this::ensureOpen, this.translogStats, new Translog.Snapshot() { - @Override - public void close() {} - - @Override - public int totalOperations() { - return 0; - } - - @Override - public Translog.Operation next() { - return null; - } - }); + translogManager = new NoOpTranslogManager(shardId, readLock, this::ensureOpen, this.translogStats, EMPTY_TRANSLOG_SNAPSHOT); success = true; } finally { diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 02f20504b07ba..f26e53967b873 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2515,8 +2515,10 @@ public void openEngineAndRecoverFromTranslog(boolean syncFromRemote) throws IOEx ); }; - // Do not load the global checkpoint if this is a remote snapshot index - if (indexSettings.isRemoteSnapshot() == false && indexSettings.isRemoteTranslogStoreEnabled() == false) { + // Do not load the global checkpoint if this is a remote snapshot index or using ingestion source + if (indexSettings.isRemoteSnapshot() == false + && indexSettings.isRemoteTranslogStoreEnabled() == false + && !indexSettings.getIndexMetadata().useIngestionSource()) { loadGlobalCheckpointToReplicationTracker(); } @@ -2635,6 +2637,9 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b // time elapses after the engine is created above (pulling the config settings) until we set the engine reference, during // which settings changes could possibly have happened, so here we forcefully push any config changes to the new engine. onSettingsChanged(); + if (indexSettings.getIndexMetadata().useIngestionSource()) { + return; + } assert assertSequenceNumbersInCommit(); recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); } @@ -4071,8 +4076,9 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro isReadOnlyReplica, this::enableUploadToRemoteTranslog, translogFactorySupplier.apply(indexSettings, shardRouting), - isTimeSeriesDescSortOptimizationEnabled() ? DataStream.TIMESERIES_LEAF_SORTER : null // DESC @timestamp default order for + isTimeSeriesDescSortOptimizationEnabled() ? DataStream.TIMESERIES_LEAF_SORTER : null, // DESC @timestamp default order for // timeseries + () -> docMapper() ); } diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index 4b4ceb7444471..ffda06d8d8292 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -959,6 +959,21 @@ public int hashCode() { } } + public static final Translog.Snapshot EMPTY_TRANSLOG_SNAPSHOT = new Translog.Snapshot() { + @Override + public void close() {} + + @Override + public int totalOperations() { + return 0; + } + + @Override + public Translog.Operation next() { + return null; + } + }; + /** * A snapshot of the transaction log, allows to iterate over all the transaction log operations. * diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 67fab720d95dd..fdec2cebd2863 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -105,6 +105,7 @@ import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; +import org.opensearch.index.IngestionConsumerFactory; import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.cache.request.ShardRequestCache; import org.opensearch.index.compositeindex.CompositeIndexSettings; @@ -147,6 +148,7 @@ import org.opensearch.indices.cluster.IndicesClusterStateService; import org.opensearch.indices.fielddata.cache.IndicesFieldDataCache; import org.opensearch.indices.mapper.MapperRegistry; +import org.opensearch.indices.pollingingest.IngestionEngineFactory; import org.opensearch.indices.recovery.PeerRecoveryTargetService; import org.opensearch.indices.recovery.RecoveryListener; import org.opensearch.indices.recovery.RecoverySettings; @@ -343,6 +345,7 @@ public class IndicesService extends AbstractLifecycleComponent private final MetaStateService metaStateService; private final Collection>> engineFactoryProviders; private final Map directoryFactories; + private final Map ingestionConsumerFactories; private final Map recoveryStateFactories; final AbstractRefCounted indicesRefCount; // pkg-private for testing private final CountDownLatch closeLatch = new CountDownLatch(1); @@ -395,6 +398,7 @@ public IndicesService( Supplier repositoriesServiceSupplier, SearchRequestStats searchRequestStats, @Nullable RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + Map ingestionConsumerFactories, RecoverySettings recoverySettings, CacheService cacheService, RemoteStoreSettings remoteStoreSettings, @@ -452,6 +456,7 @@ public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, lon this.directoryFactories = directoryFactories; this.recoveryStateFactories = recoveryStateFactories; + this.ingestionConsumerFactories = ingestionConsumerFactories; // doClose() is called when shutting down a node, yet there might still be ongoing requests // that we need to wait for before closing some resources such as the caches. In order to // avoid closing these resources while ongoing requests are still being processed, we use a @@ -540,6 +545,7 @@ public IndicesService( Supplier repositoriesServiceSupplier, SearchRequestStats searchRequestStats, @Nullable RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + Map ingestionConsumerFactories, RecoverySettings recoverySettings, CacheService cacheService, RemoteStoreSettings remoteStoreSettings @@ -569,6 +575,7 @@ public IndicesService( repositoriesServiceSupplier, searchRequestStats, remoteStoreStatsTrackerFactory, + ingestionConsumerFactories, recoverySettings, cacheService, remoteStoreSettings, @@ -999,6 +1006,21 @@ private EngineConfigFactory getEngineConfigFactory(final IndexSettings idxSettin return new EngineConfigFactory(this.pluginsService, idxSettings); } + private IngestionConsumerFactory getIngestionConsumerFactory(final IndexSettings idxSettings) { + final IndexMetadata indexMetadata = idxSettings.getIndexMetadata(); + if (indexMetadata == null) { + return null; + } + if (indexMetadata.useIngestionSource()) { + String type = indexMetadata.getIngestionSource().getType().toUpperCase(Locale.ROOT); + if (!ingestionConsumerFactories.containsKey(type)) { + throw new IllegalArgumentException("No factory found for ingestion source type [" + type + "]"); + } + return ingestionConsumerFactories.get(type); + } + return null; + } + private EngineFactory getEngineFactory(final IndexSettings idxSettings) { final IndexMetadata indexMetadata = idxSettings.getIndexMetadata(); if (indexMetadata != null && indexMetadata.getState() == IndexMetadata.State.CLOSE) { @@ -1006,6 +1028,12 @@ private EngineFactory getEngineFactory(final IndexSettings idxSettings) { return NoOpEngine::new; } + // streaming ingestion + if (indexMetadata != null && indexMetadata.useIngestionSource()) { + IngestionConsumerFactory ingestionConsumerFactory = getIngestionConsumerFactory(idxSettings); + return new IngestionEngineFactory(ingestionConsumerFactory); + } + final List> engineFactories = engineFactoryProviders.stream() .map(engineFactoryProvider -> engineFactoryProvider.apply(idxSettings)) .filter(maybe -> Objects.requireNonNull(maybe).isPresent()) diff --git a/server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java b/server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java new file mode 100644 index 0000000000000..b5c1db999544a --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/pollingingest/DefaultStreamPoller.java @@ -0,0 +1,286 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.Nullable; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.Message; +import org.opensearch.index.engine.IngestionEngine; + +import java.util.List; +import java.util.Locale; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * Default implementation of {@link StreamPoller} + */ +public class DefaultStreamPoller implements StreamPoller { + private static final Logger logger = LogManager.getLogger(DefaultStreamPoller.class); + + // TODO: make this configurable + public static final long MAX_POLL_SIZE = 1000; + public static final int POLL_TIMEOUT = 1000; + + private volatile State state = State.NONE; + + // goal state + private volatile boolean started; + private volatile boolean closed; + private volatile boolean paused; + + private IngestionShardConsumer consumer; + + private ExecutorService consumerThread; + + private ExecutorService processorThread; + + // start of the batch, inclusive + private IngestionShardPointer batchStartPointer; + + private ResetState resetState; + + private Set persistedPointers; + + private BlockingQueue> blockingQueue; + + private MessageProcessorRunnable processorRunnable; + + // A pointer to the max persisted pointer for optimizing the check + @Nullable + private IngestionShardPointer maxPersistedPointer; + + public DefaultStreamPoller( + IngestionShardPointer startPointer, + Set persistedPointers, + IngestionShardConsumer consumer, + IngestionEngine ingestionEngine, + ResetState resetState + ) { + this( + startPointer, + persistedPointers, + consumer, + new MessageProcessorRunnable(new ArrayBlockingQueue<>(100), ingestionEngine), + resetState + ); + } + + DefaultStreamPoller( + IngestionShardPointer startPointer, + Set persistedPointers, + IngestionShardConsumer consumer, + MessageProcessorRunnable processorRunnable, + ResetState resetState + ) { + this.consumer = Objects.requireNonNull(consumer); + this.resetState = resetState; + batchStartPointer = startPointer; + this.persistedPointers = persistedPointers; + if (!this.persistedPointers.isEmpty()) { + maxPersistedPointer = this.persistedPointers.stream().max(IngestionShardPointer::compareTo).get(); + } + this.processorRunnable = processorRunnable; + blockingQueue = processorRunnable.getBlockingQueue(); + this.consumerThread = Executors.newSingleThreadExecutor( + r -> new Thread( + r, + String.format(Locale.ROOT, "stream-poller-consumer-%d-%d", consumer.getShardId(), System.currentTimeMillis()) + ) + ); + + // TODO: allow multiple threads for processing the messages in parallel + this.processorThread = Executors.newSingleThreadExecutor( + r -> new Thread( + r, + String.format(Locale.ROOT, "stream-poller-processor-%d-%d", consumer.getShardId(), System.currentTimeMillis()) + ) + ); + } + + @Override + public void start() { + if (closed) { + throw new RuntimeException("poller is closed!"); + } + started = true; + consumerThread.submit(this::startPoll); + processorThread.submit(processorRunnable); + } + + /** + * Start the poller. visibile for testing + */ + protected void startPoll() { + if (!started) { + throw new IllegalStateException("poller is not started!"); + } + if (closed) { + throw new IllegalStateException("poller is closed!"); + } + logger.info("Starting poller for shard {}", consumer.getShardId()); + + while (true) { + try { + if (closed) { + state = State.CLOSED; + break; + } + + // reset the offset + if (resetState != ResetState.NONE) { + switch (resetState) { + case EARLIEST: + batchStartPointer = consumer.earliestPointer(); + logger.info("Resetting offset by seeking to earliest offset {}", batchStartPointer.asString()); + break; + case LATEST: + batchStartPointer = consumer.latestPointer(); + logger.info("Resetting offset by seeking to latest offset {}", batchStartPointer.asString()); + break; + } + resetState = ResetState.NONE; + } + + if (paused) { + state = State.PAUSED; + try { + // TODO: make sleep time configurable + Thread.sleep(100); + } catch (Throwable e) { + logger.error("Error in pausing the poller of shard {}: {}", consumer.getShardId(), e); + } + continue; + } + + state = State.POLLING; + + List> results = consumer.readNext( + batchStartPointer, + MAX_POLL_SIZE, + POLL_TIMEOUT + ); + + if (results.isEmpty()) { + // no new records + continue; + } + + state = State.PROCESSING; + // process the records + for (IngestionShardConsumer.ReadResult result : results) { + // check if the message is already processed + if (isProcessed(result.getPointer())) { + logger.info("Skipping message with pointer {} as it is already processed", result.getPointer().asString()); + continue; + } + blockingQueue.put(result); + logger.debug( + "Put message {} with pointer {} to the blocking queue", + String.valueOf(result.getMessage().getPayload()), + result.getPointer().asString() + ); + } + // update the batch start pointer to the next batch + batchStartPointer = consumer.nextPointer(); + } catch (Throwable e) { + // TODO better error handling + logger.error("Error in polling the shard {}: {}", consumer.getShardId(), e); + } + } + } + + private boolean isProcessed(IngestionShardPointer pointer) { + if (maxPersistedPointer == null) { + return false; + } + if (pointer.compareTo(maxPersistedPointer) > 0) { + return false; + } + return persistedPointers.contains(pointer); + } + + /** + * Visible for testing. Get the max persisted pointer + * @return the max persisted pointer + */ + protected IngestionShardPointer getMaxPersistedPointer() { + return maxPersistedPointer; + } + + @Override + public void pause() { + if (closed) { + throw new RuntimeException("consumer is closed!"); + } + paused = true; + } + + @Override + public void resume() { + if (closed) { + throw new RuntimeException("consumer is closed!"); + } + paused = false; + } + + @Override + public void close() { + closed = true; + if (!started) { + logger.info("consumer thread not started"); + return; + } + long startTime = System.currentTimeMillis(); // Record the start time + long timeout = 5000; + while (state != State.CLOSED) { + // Check if the timeout has been reached + if (System.currentTimeMillis() - startTime > timeout) { + logger.error("Timeout reached while waiting for shard {} to close", consumer.getShardId()); + break; // Exit the loop if the timeout is reached + } + try { + Thread.sleep(100); + } catch (Throwable e) { + logger.error("Error in closing the poller of shard {}: {}", consumer.getShardId(), e); + } + } + blockingQueue.clear(); + consumerThread.shutdown(); + // interrupts the processor + processorThread.shutdownNow(); + logger.info("closed the poller of shard {}", consumer.getShardId()); + } + + @Override + public boolean isPaused() { + return paused; + } + + @Override + public boolean isClosed() { + return closed; + } + + @Override + public IngestionShardPointer getBatchStartPointer() { + return batchStartPointer; + } + + public State getState() { + return state; + } +} diff --git a/server/src/main/java/org/opensearch/indices/pollingingest/IngestionEngineFactory.java b/server/src/main/java/org/opensearch/indices/pollingingest/IngestionEngineFactory.java new file mode 100644 index 0000000000000..e124adb90365b --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/pollingingest/IngestionEngineFactory.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.opensearch.index.IngestionConsumerFactory; +import org.opensearch.index.engine.Engine; +import org.opensearch.index.engine.EngineConfig; +import org.opensearch.index.engine.EngineFactory; +import org.opensearch.index.engine.IngestionEngine; + +import java.util.Objects; + +/** + * Engine Factory implementation used with streaming ingestion. + */ +public class IngestionEngineFactory implements EngineFactory { + + private final IngestionConsumerFactory ingestionConsumerFactory; + + public IngestionEngineFactory(IngestionConsumerFactory ingestionConsumerFactory) { + this.ingestionConsumerFactory = Objects.requireNonNull(ingestionConsumerFactory); + } + + @Override + public Engine newReadWriteEngine(EngineConfig config) { + IngestionEngine ingestionEngine = new IngestionEngine(config, ingestionConsumerFactory); + ingestionEngine.start(); + return ingestionEngine; + } +} diff --git a/server/src/main/java/org/opensearch/indices/pollingingest/MessageProcessorRunnable.java b/server/src/main/java/org/opensearch/indices/pollingingest/MessageProcessorRunnable.java new file mode 100644 index 0000000000000..53f9353477869 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/pollingingest/MessageProcessorRunnable.java @@ -0,0 +1,236 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.index.Term; +import org.opensearch.action.DocWriteRequest; +import org.opensearch.common.lucene.uid.Versions; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.Message; +import org.opensearch.index.VersionType; +import org.opensearch.index.engine.Engine; +import org.opensearch.index.engine.IngestionEngine; +import org.opensearch.index.mapper.IdFieldMapper; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.index.mapper.ParsedDocument; +import org.opensearch.index.mapper.SourceToParse; +import org.opensearch.index.mapper.Uid; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.TimeUnit; + +import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; + +/** + * A class to process messages from the ingestion stream. It extracts the payload from the message and creates an + * engine operation. + */ +public class MessageProcessorRunnable implements Runnable { + private static final Logger logger = LogManager.getLogger(MessageProcessorRunnable.class); + + private final BlockingQueue> blockingQueue; + private final MessageProcessor messageProcessor; + + private static final String ID = "_id"; + private static final String OP_TYPE = "_op_type"; + private static final String SOURCE = "_source"; + + /** + * Constructor. + * + * @param blockingQueue the blocking queue to poll messages from + * @param engine the ingestion engine + */ + public MessageProcessorRunnable( + BlockingQueue> blockingQueue, + IngestionEngine engine + ) { + this(blockingQueue, new MessageProcessor(engine)); + } + + /** + * Constructor visible for testing. + * @param blockingQueue the blocking queue to poll messages from + * @param messageProcessor the message processor + */ + MessageProcessorRunnable( + BlockingQueue> blockingQueue, + MessageProcessor messageProcessor + ) { + this.blockingQueue = Objects.requireNonNull(blockingQueue); + this.messageProcessor = messageProcessor; + } + + static class MessageProcessor { + private final IngestionEngine engine; + private final String index; + + MessageProcessor(IngestionEngine engine) { + this(engine, engine.config().getIndexSettings().getIndex().getName()); + } + + /** + * visible for testing + * @param engine the ingestion engine + * @param index the index name + */ + MessageProcessor(IngestionEngine engine, String index) { + this.engine = engine; + this.index = index; + } + + /** + * Visible for testing. Process the message and create an engine operation. + * + * Process the message and create an engine operation. It also records the offset in the document as (1) a point + * field used for range search, (2) a stored field for retrieval. + * + * @param message the message to process + * @param pointer the pointer to the message + */ + protected void process(Message message, IngestionShardPointer pointer) { + byte[] payload = (byte[]) message.getPayload(); + + try { + Engine.Operation operation = getOperation(payload, pointer); + switch (operation.operationType()) { + case INDEX: + engine.index((Engine.Index) operation); + break; + case DELETE: + engine.delete((Engine.Delete) operation); + break; + default: + throw new IllegalArgumentException("Invalid operation: " + operation); + } + } catch (IOException e) { + logger.error("Failed to process operation from message {} at pointer {}: {}", message, pointer, e); + throw new RuntimeException(e); + } + } + + /** + * Visible for testing. Get the engine operation from the message. + * @param payload the payload of the message + * @param pointer the pointer to the message + * @return the engine operation + */ + protected Engine.Operation getOperation(byte[] payload, IngestionShardPointer pointer) throws IOException { + BytesReference payloadBR = new BytesArray(payload); + Map payloadMap = XContentHelper.convertToMap(payloadBR, false, MediaTypeRegistry.xContentType(payloadBR)).v2(); + + String id = (String) payloadMap.getOrDefault(ID, "null"); + if (payloadMap.containsKey(OP_TYPE) && !(payloadMap.get(OP_TYPE) instanceof String)) { + // TODO: add metric + logger.error("_op_type field is of type {} but not string, skipping the message", payloadMap.get(OP_TYPE).getClass()); + return null; + } + String opTypeString = (String) payloadMap.getOrDefault(OP_TYPE, "index"); + DocWriteRequest.OpType opType = DocWriteRequest.OpType.fromString(opTypeString); + + Engine.Operation operation; + switch (opType) { + case INDEX: + if (!payloadMap.containsKey(SOURCE)) { + // TODO: add metric + logger.error("missing _source field, skipping the message"); + return null; + } + if (!(payloadMap.get(SOURCE) instanceof Map)) { + // TODO: add metric + logger.error("_source field does not contain a map, skipping the message"); + return null; + } + BytesReference source = convertToBytes(payloadMap.get(SOURCE)); + + SourceToParse sourceToParse = new SourceToParse(index, id, source, MediaTypeRegistry.xContentType(source), null); + // TODO: handle parsing err + ParsedDocument doc = engine.getDocumentMapperForType().getDocumentMapper().parse(sourceToParse); + ParseContext.Document document = doc.rootDoc(); + // set the offset as the offset field + document.add(pointer.asPointField(IngestionShardPointer.OFFSET_FIELD)); + // store the offset as string in stored field + document.add(new StoredField(IngestionShardPointer.OFFSET_FIELD, pointer.asString())); + + operation = new Engine.Index( + new Term("_id", id), + doc, + 0, + 1, + Versions.MATCH_ANY, + VersionType.INTERNAL, + Engine.Operation.Origin.PRIMARY, + System.nanoTime(), + System.currentTimeMillis(), + false, + UNASSIGNED_SEQ_NO, + 0 + ); + break; + case DELETE: + operation = new Engine.Delete( + id, + new Term(IdFieldMapper.NAME, Uid.encodeId(id)), + 0, + 1, + Versions.MATCH_ANY, + VersionType.INTERNAL, + Engine.Operation.Origin.PRIMARY, + System.nanoTime(), + UNASSIGNED_SEQ_NO, + 0 + ); + break; + default: + logger.error("Unsupported operation type {}", opType); + return null; + } + + return operation; + } + } + + private static BytesReference convertToBytes(Object object) throws IOException { + assert object instanceof Map; + return BytesReference.bytes(XContentFactory.jsonBuilder().map((Map) object)); + } + + BlockingQueue> getBlockingQueue() { + return blockingQueue; + } + + @Override + public void run() { + while (!(Thread.currentThread().isInterrupted())) { + IngestionShardConsumer.ReadResult result = null; + try { + result = blockingQueue.poll(1000, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // TODO: add metric + logger.debug("MessageProcessorRunnable poll interruptedException", e); + Thread.currentThread().interrupt(); // Restore interrupt status + } + if (result != null) { + messageProcessor.process(result.getMessage(), result.getPointer()); + } + } + } +} diff --git a/server/src/main/java/org/opensearch/indices/pollingingest/StreamPoller.java b/server/src/main/java/org/opensearch/indices/pollingingest/StreamPoller.java new file mode 100644 index 0000000000000..f674f6dc55c85 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/pollingingest/StreamPoller.java @@ -0,0 +1,71 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.opensearch.index.IngestionShardPointer; + +import java.io.Closeable; + +/** + * A poller for reading messages from an ingestion shard. This is used in the ingestion engine. + */ +public interface StreamPoller extends Closeable { + + String BATCH_START = "batch_start"; + + /** + * Start the poller + */ + void start();; + + /** + * Pause the poller + */ + void pause(); + + /** + * Resume the poller polling + */ + void resume(); + + /** + * @return if the poller is paused + */ + boolean isPaused(); + + /** + * check if the poller is closed + */ + boolean isClosed(); + + /** + * get the pointer to the start of the current batch of messages. + */ + IngestionShardPointer getBatchStartPointer(); + + /** + * a state to indicate the current state of the poller + */ + enum State { + NONE, + CLOSED, + PAUSED, + POLLING, + PROCESSING, + } + + /** + * a reset state to indicate how to reset the pointer + */ + enum ResetState { + EARLIEST, + LATEST, + NONE, + } +} diff --git a/server/src/main/java/org/opensearch/indices/pollingingest/package-info.java b/server/src/main/java/org/opensearch/indices/pollingingest/package-info.java new file mode 100644 index 0000000000000..1c3d860bf2559 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/pollingingest/package-info.java @@ -0,0 +1,10 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +/** Indices ingestion module package. */ +package org.opensearch.indices.pollingingest; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index c2c4e68dae257..622b8f7bfc565 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -150,6 +150,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexingPressureService; +import org.opensearch.index.IngestionConsumerFactory; import org.opensearch.index.SegmentReplicationStatsTracker; import org.opensearch.index.analysis.AnalysisRegistry; import org.opensearch.index.compositeindex.CompositeIndexSettings; @@ -207,6 +208,7 @@ import org.opensearch.plugins.IdentityPlugin; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.plugins.IngestPlugin; +import org.opensearch.plugins.IngestionConsumerPlugin; import org.opensearch.plugins.MapperPlugin; import org.opensearch.plugins.MetadataUpgrader; import org.opensearch.plugins.NetworkPlugin; @@ -859,6 +861,11 @@ protected Node( .map(plugin -> (Function>) plugin::getEngineFactory) .collect(Collectors.toList()); + // collect ingestion consumer factory providers from plugins + final Map ingestionConsumerFactories = new HashMap<>(); + pluginsService.filterPlugins(IngestionConsumerPlugin.class) + .forEach(plugin -> ingestionConsumerFactories.putAll(plugin.getIngestionConsumerFactories())); + final Map builtInDirectoryFactories = IndexModule.createBuiltInDirectoryFactories( repositoriesServiceReference::get, threadPool, @@ -944,6 +951,7 @@ protected Node( repositoriesServiceReference::get, searchRequestStats, remoteStoreStatsTrackerFactory, + ingestionConsumerFactories, recoverySettings, cacheService, remoteStoreSettings, diff --git a/server/src/main/java/org/opensearch/plugins/IngestionConsumerPlugin.java b/server/src/main/java/org/opensearch/plugins/IngestionConsumerPlugin.java new file mode 100644 index 0000000000000..e4b9889b5bfa7 --- /dev/null +++ b/server/src/main/java/org/opensearch/plugins/IngestionConsumerPlugin.java @@ -0,0 +1,36 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.plugins; + +import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.index.IngestionConsumerFactory; + +import java.util.Map; + +/** + * An extension point for {@link Plugin} implementations to add custom ingestion consumers for the {@link org.opensearch.index.engine.IngestionEngine} + * + * @opensearch.api + */ +@ExperimentalApi +public interface IngestionConsumerPlugin { + + /** + * When an ingestion index is created this method is invoked for each ingestion consumer plugin. + * Ingestion consumer plugins can inspect the index settings to determine which ingestion consumer to provide. + * + * @return a map from the ingestion consumer type to the factory + */ + Map getIngestionConsumerFactories(); + + /** + * @return the type of the ingestion consumer plugin. the type name shall be in upper case + */ + String getType(); +} diff --git a/server/src/main/resources/org/opensearch/bootstrap/test-framework.policy b/server/src/main/resources/org/opensearch/bootstrap/test-framework.policy index e1a3b4618035e..78f302e9b23db 100644 --- a/server/src/main/resources/org/opensearch/bootstrap/test-framework.policy +++ b/server/src/main/resources/org/opensearch/bootstrap/test-framework.policy @@ -167,4 +167,5 @@ grant { permission org.opensearch.secure_sm.ThreadContextPermission "markAsSystemContext"; permission org.opensearch.secure_sm.ThreadContextPermission "stashAndMergeHeaders"; permission org.opensearch.secure_sm.ThreadContextPermission "stashWithOrigin"; + permission java.lang.RuntimePermission "setDefaultUncaughtExceptionHandler"; }; diff --git a/server/src/test/java/org/opensearch/cluster/metadata/IngestionSourceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/IngestionSourceTests.java new file mode 100644 index 0000000000000..f67d13e54e608 --- /dev/null +++ b/server/src/test/java/org/opensearch/cluster/metadata/IngestionSourceTests.java @@ -0,0 +1,67 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.metadata; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +public class IngestionSourceTests extends OpenSearchTestCase { + + public void testConstructorAndGetters() { + Map params = new HashMap<>(); + params.put("key", "value"); + IngestionSource source = new IngestionSource("type", "pointerInitReset", params); + + assertEquals("type", source.getType()); + assertEquals("pointerInitReset", source.getPointerInitReset()); + assertEquals(params, source.params()); + } + + public void testEquals() { + Map params1 = new HashMap<>(); + params1.put("key", "value"); + IngestionSource source1 = new IngestionSource("type", "pointerInitReset", params1); + + Map params2 = new HashMap<>(); + params2.put("key", "value"); + IngestionSource source2 = new IngestionSource("type", "pointerInitReset", params2); + + assertTrue(source1.equals(source2)); + assertTrue(source2.equals(source1)); + + IngestionSource source3 = new IngestionSource("differentType", "pointerInitReset", params1); + assertFalse(source1.equals(source3)); + } + + public void testHashCode() { + Map params1 = new HashMap<>(); + params1.put("key", "value"); + IngestionSource source1 = new IngestionSource("type", "pointerInitReset", params1); + + Map params2 = new HashMap<>(); + params2.put("key", "value"); + IngestionSource source2 = new IngestionSource("type", "pointerInitReset", params2); + + assertEquals(source1.hashCode(), source2.hashCode()); + + IngestionSource source3 = new IngestionSource("differentType", "pointerInitReset", params1); + assertNotEquals(source1.hashCode(), source3.hashCode()); + } + + public void testToString() { + Map params = new HashMap<>(); + params.put("key", "value"); + IngestionSource source = new IngestionSource("type", "pointerInitReset", params); + + String expected = "IngestionSource{type='type',pointer_init_reset='pointerInitReset', params={key=value}}"; + assertEquals(expected, source.toString()); + } +} diff --git a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java index bf9a86cff8b76..f8ed68eb2e0a3 100644 --- a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java @@ -70,6 +70,7 @@ public void testCreateEngineConfigFromFactory() { false, () -> Boolean.TRUE, new InternalTranslogFactory(), + null, null ); @@ -150,6 +151,7 @@ public void testCreateCodecServiceFromFactory() { false, () -> Boolean.TRUE, new InternalTranslogFactory(), + null, null ); assertNotNull(config.getCodec()); diff --git a/server/src/test/java/org/opensearch/index/engine/FakeIngestionSource.java b/server/src/test/java/org/opensearch/index/engine/FakeIngestionSource.java new file mode 100644 index 0000000000000..de03dcd313c29 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/engine/FakeIngestionSource.java @@ -0,0 +1,183 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.engine; + +import org.apache.lucene.document.Field; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.search.Query; +import org.opensearch.index.IngestionConsumerFactory; +import org.opensearch.index.IngestionShardConsumer; +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.Message; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; + +/** + * A fake ingestion source for testing purposes. + */ +public class FakeIngestionSource { + + public static class FakeIngestionConsumerFactory implements IngestionConsumerFactory { + private List messages; + + public FakeIngestionConsumerFactory(List messages) { + this.messages = messages; + } + + @Override + public void initialize(Map params) {} + + @Override + public FakeIngestionConsumer createShardConsumer(String clientId, int shardId) { + return new FakeIngestionConsumer(messages, shardId); + } + + @Override + public FakeIngestionShardPointer parsePointerFromString(String pointer) { + return new FakeIngestionShardPointer(Long.valueOf(pointer)); + } + } + + public static class FakeIngestionConsumer implements IngestionShardConsumer { + // FakeIngestionConsumer uses a list of byte arrays to simulate streams + private List messages; + private int shardId; + private long lastFetchedOffset; + + public FakeIngestionConsumer(List messages, int shardId) { + this.messages = messages; + this.shardId = shardId; + this.lastFetchedOffset = -1; + } + + @Override + public List> readNext( + FakeIngestionShardPointer pointer, + long maxMessages, + int timeoutMillis + ) throws TimeoutException { + lastFetchedOffset = pointer.offset - 1; + int numToFetch = Math.min(messages.size() - (int) pointer.offset, (int) maxMessages); + List> result = new ArrayList<>(); + for (long i = pointer.offset; i < pointer.offset + numToFetch; i++) { + result.add(new ReadResult<>(new FakeIngestionShardPointer(i), new FakeIngestionMessage(messages.get((int) i)))); + lastFetchedOffset = i; + } + return result; + } + + @Override + public FakeIngestionShardPointer nextPointer() { + return new FakeIngestionShardPointer(lastFetchedOffset + 1); + } + + @Override + public FakeIngestionShardPointer earliestPointer() { + return new FakeIngestionShardPointer(0); + } + + @Override + public FakeIngestionShardPointer latestPointer() { + return new FakeIngestionShardPointer(messages.size()); + } + + @Override + public int getShardId() { + return shardId; + } + + @Override + public void close() throws IOException { + + } + } + + public static class FakeIngestionMessage implements Message { + private final byte[] payload; + + public FakeIngestionMessage(byte[] payload) { + this.payload = payload; + } + + @Override + public byte[] getPayload() { + return payload; + } + + @Override + public String toString() { + return new String(payload, StandardCharsets.UTF_8); + } + } + + public static class FakeIngestionShardPointer implements IngestionShardPointer { + private final long offset; + + public FakeIngestionShardPointer(long offset) { + this.offset = offset; + } + + @Override + public byte[] serialize() { + ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES); + buffer.putLong(offset); + return buffer.array(); + } + + @Override + public String asString() { + return String.valueOf(offset); + } + + @Override + public String toString() { + return asString(); + } + + @Override + public Field asPointField(String fieldName) { + return new LongPoint(fieldName, offset); + } + + @Override + public Query newRangeQueryGreaterThan(String fieldName) { + return LongPoint.newRangeQuery(fieldName, offset, Long.MAX_VALUE); + } + + @Override + public int compareTo(IngestionShardPointer o) { + FakeIngestionShardPointer other = (FakeIngestionShardPointer) o; + return Long.compare(offset, other.offset); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + FakeIngestionShardPointer that = (FakeIngestionShardPointer) o; + return offset == that.offset; + } + + @Override + public int hashCode() { + return Long.hashCode(offset); + } + } + +} diff --git a/server/src/test/java/org/opensearch/index/engine/IngestionEngineTests.java b/server/src/test/java/org/opensearch/index/engine/IngestionEngineTests.java new file mode 100644 index 0000000000000..19718384bd926 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/engine/IngestionEngineTests.java @@ -0,0 +1,189 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.engine; + +import org.apache.lucene.index.NoMergePolicy; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.lucene.Lucene; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.mapper.DocumentMapperForType; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.Store; +import org.opensearch.index.translog.Translog; +import org.opensearch.indices.pollingingest.StreamPoller; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.IndexSettingsModule; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import static org.awaitility.Awaitility.await; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; + +public class IngestionEngineTests extends EngineTestCase { + + private IndexSettings indexSettings; + private Store ingestionEngineStore; + private IngestionEngine ingestionEngine; + // the messages of the stream to ingest from + private List messages; + + @Override + @Before + public void setUp() throws Exception { + indexSettings = newIndexSettings(); + super.setUp(); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + ingestionEngineStore = createStore(indexSettings, newDirectory()); + // create some initial messages + messages = new ArrayList<>(); + publishData("{\"_id\":\"2\",\"_source\":{\"name\":\"bob\", \"age\": 24}}"); + publishData("{\"_id\":\"1\",\"_source\":{\"name\":\"alice\", \"age\": 20}}"); + ingestionEngine = buildIngestionEngine(globalCheckpoint, ingestionEngineStore, indexSettings); + } + + private void publishData(String message) { + messages.add(message.getBytes(StandardCharsets.UTF_8)); + } + + protected IndexSettings newIndexSettings() { + return IndexSettingsModule.newIndexSettings( + "index", + Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_INGESTION_SOURCE_TYPE, "fake") + .put(IndexMetadata.SETTING_INGESTION_SOURCE_POINTER_INIT_RESET, "earliest") + .build() + ); + } + + @Override + @After + public void tearDown() throws Exception { + if (ingestionEngine != null) { + ingestionEngine.close(); + } + if (ingestionEngineStore != null) { + ingestionEngineStore.close(); + } + super.tearDown(); + } + + public void testCreateEngine() throws IOException { + // wait for the engine to ingest all messages + waitForResults(ingestionEngine, 2); + // flush + ingestionEngine.flush(false, true); + Map commitData = ingestionEngine.commitDataAsMap(); + // verify the commit data + Assert.assertEquals(1, commitData.size()); + Assert.assertEquals("2", commitData.get(StreamPoller.BATCH_START)); + + // verify the stored offsets + var offset = new FakeIngestionSource.FakeIngestionShardPointer(0); + ingestionEngine.refresh("read_offset"); + try (Engine.Searcher searcher = ingestionEngine.acquireSearcher("read_offset")) { + Set persistedPointers = ingestionEngine.fetchPersistedOffsets( + Lucene.wrapAllDocsLive(searcher.getDirectoryReader()), + offset + ); + Assert.assertEquals(2, persistedPointers.size()); + } + } + + public void testRecovery() throws IOException { + // wait for the engine to ingest all messages + waitForResults(ingestionEngine, 2); + // flush + ingestionEngine.flush(false, true); + + // ingest some new messages + publishData("{\"_id\":\"3\",\"_source\":{\"name\":\"john\", \"age\": 30}}"); + publishData("{\"_id\":\"4\",\"_source\":{\"name\":\"jane\", \"age\": 25}}"); + ingestionEngine.close(); + ingestionEngine = buildIngestionEngine(new AtomicLong(2), ingestionEngineStore, indexSettings); + waitForResults(ingestionEngine, 4); + } + + public void testCreationFailure() throws IOException { + // Simulate an error scenario + Store mockStore = mock(Store.class); + doThrow(new IOException("Simulated IOException")).when(mockStore).readLastCommittedSegmentsInfo(); + + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + + FakeIngestionSource.FakeIngestionConsumerFactory consumerFactory = new FakeIngestionSource.FakeIngestionConsumerFactory(messages); + EngineConfig engineConfig = config( + indexSettings, + store, + createTempDir(), + NoMergePolicy.INSTANCE, + null, + null, + globalCheckpoint::get + ); + // overwrite the config with ingestion engine settings + String mapping = "{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}"; + MapperService mapperService = createMapperService(mapping); + engineConfig = config(engineConfig, () -> new DocumentMapperForType(mapperService.documentMapper(), null)); + try { + new IngestionEngine(engineConfig, consumerFactory); + fail("Expected EngineException to be thrown"); + } catch (EngineException e) { + assertEquals("failed to create engine", e.getMessage()); + assertTrue(e.getCause() instanceof IOException); + } + } + + private IngestionEngine buildIngestionEngine(AtomicLong globalCheckpoint, Store store, IndexSettings settings) throws IOException { + FakeIngestionSource.FakeIngestionConsumerFactory consumerFactory = new FakeIngestionSource.FakeIngestionConsumerFactory(messages); + EngineConfig engineConfig = config(settings, store, createTempDir(), NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); + // overwrite the config with ingestion engine settings + String mapping = "{\"properties\":{\"name\":{\"type\": \"text\"},\"age\":{\"type\": \"integer\"}}}}"; + MapperService mapperService = createMapperService(mapping); + engineConfig = config(engineConfig, () -> new DocumentMapperForType(mapperService.documentMapper(), null)); + if (!Lucene.indexExists(store.directory())) { + store.createEmpty(engineConfig.getIndexSettings().getIndexVersionCreated().luceneVersion); + final String translogUuid = Translog.createEmptyTranslog( + engineConfig.getTranslogConfig().getTranslogPath(), + SequenceNumbers.NO_OPS_PERFORMED, + shardId, + primaryTerm.get() + ); + store.associateIndexWithNewTranslog(translogUuid); + } + IngestionEngine ingestionEngine = new IngestionEngine(engineConfig, consumerFactory); + ingestionEngine.start(); + return ingestionEngine; + } + + private void waitForResults(Engine engine, int numDocs) { + await().atMost(3, TimeUnit.SECONDS).untilAsserted(() -> { Assert.assertTrue(resultsFound(engine, numDocs)); }); + } + + private boolean resultsFound(Engine engine, int numDocs) { + engine.refresh("index"); + try (Engine.Searcher searcher = engine.acquireSearcher("index")) { + return searcher.getIndexReader().numDocs() == numDocs; + } + } +} diff --git a/server/src/test/java/org/opensearch/indices/pollingingest/DefaultStreamPollerTests.java b/server/src/test/java/org/opensearch/indices/pollingingest/DefaultStreamPollerTests.java new file mode 100644 index 0000000000000..1a98f65d04f7c --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/pollingingest/DefaultStreamPollerTests.java @@ -0,0 +1,168 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.opensearch.index.IngestionShardPointer; +import org.opensearch.index.engine.FakeIngestionSource; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.After; +import org.junit.Before; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class DefaultStreamPollerTests extends OpenSearchTestCase { + private DefaultStreamPoller poller; + private FakeIngestionSource.FakeIngestionConsumer fakeConsumer; + private MessageProcessorRunnable processorRunnable; + private MessageProcessorRunnable.MessageProcessor processor; + private List messages; + private Set persistedPointers; + private final int sleepTime = 300; + + @Before + public void setUp() throws Exception { + super.setUp(); + messages = new ArrayList<>(); + ; + messages.add("{\"_id\":\"1\",\"_source\":{\"name\":\"bob\", \"age\": 24}}".getBytes(StandardCharsets.UTF_8)); + messages.add("{\"_id\":\"2\",\"_source\":{\"name\":\"alice\", \"age\": 21}}".getBytes(StandardCharsets.UTF_8)); + fakeConsumer = new FakeIngestionSource.FakeIngestionConsumer(messages, 0); + processor = mock(MessageProcessorRunnable.MessageProcessor.class); + processorRunnable = new MessageProcessorRunnable(new ArrayBlockingQueue<>(5), processor); + persistedPointers = new HashSet<>(); + poller = new DefaultStreamPoller( + new FakeIngestionSource.FakeIngestionShardPointer(0), + persistedPointers, + fakeConsumer, + processorRunnable, + StreamPoller.ResetState.NONE + ); + } + + @After + public void tearDown() throws Exception { + if (!poller.isClosed()) { + poller.close(); + } + super.tearDown(); + } + + public void testPauseAndResume() throws InterruptedException { + poller.pause(); + poller.start(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + assertEquals(DefaultStreamPoller.State.PAUSED, poller.getState()); + assertTrue(poller.isPaused()); + // no messages are processed + verify(processor, never()).process(any(), any()); + + poller.resume(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + assertFalse(poller.isPaused()); + // 2 messages are processed + verify(processor, times(2)).process(any(), any()); + } + + public void testSkipProcessed() throws InterruptedException { + messages.add("{\"name\":\"cathy\", \"age\": 21}".getBytes(StandardCharsets.UTF_8)); + messages.add("{\"name\":\"danny\", \"age\": 31}".getBytes(StandardCharsets.UTF_8)); + persistedPointers.add(new FakeIngestionSource.FakeIngestionShardPointer(1)); + persistedPointers.add(new FakeIngestionSource.FakeIngestionShardPointer(2)); + poller = new DefaultStreamPoller( + new FakeIngestionSource.FakeIngestionShardPointer(0), + persistedPointers, + fakeConsumer, + processorRunnable, + StreamPoller.ResetState.NONE + ); + poller.start(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + // 2 messages are processed, 2 messages are skipped + verify(processor, times(2)).process(any(), any()); + assertEquals(new FakeIngestionSource.FakeIngestionShardPointer(2), poller.getMaxPersistedPointer()); + } + + public void testCloseWithoutStart() { + poller.close(); + assertTrue(poller.isClosed()); + } + + public void testClose() throws InterruptedException { + poller.start(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + poller.close(); + assertTrue(poller.isClosed()); + assertEquals(DefaultStreamPoller.State.CLOSED, poller.getState()); + } + + public void testResetStateEarliest() throws InterruptedException { + poller = new DefaultStreamPoller( + new FakeIngestionSource.FakeIngestionShardPointer(1), + persistedPointers, + fakeConsumer, + processorRunnable, + StreamPoller.ResetState.EARLIEST + ); + + poller.start(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + + // 2 messages are processed + verify(processor, times(2)).process(any(), any()); + } + + public void testResetStateLatest() throws InterruptedException { + poller = new DefaultStreamPoller( + new FakeIngestionSource.FakeIngestionShardPointer(0), + persistedPointers, + fakeConsumer, + processorRunnable, + StreamPoller.ResetState.LATEST + ); + + poller.start(); + Thread.sleep(sleepTime); // Allow some time for the poller to run + // no messages processed + verify(processor, never()).process(any(), any()); + // reset to the latest + assertEquals(new FakeIngestionSource.FakeIngestionShardPointer(2), poller.getBatchStartPointer()); + } + + public void testStartPollWithoutStart() { + try { + poller.startPoll(); + fail("Expected an exception to be thrown"); + } catch (IllegalStateException e) { + assertEquals("poller is not started!", e.getMessage()); + } + } + + public void testStartClosedPoller() throws InterruptedException { + poller.start(); + Thread.sleep(sleepTime); + poller.close(); + try { + poller.startPoll(); + fail("Expected an exception to be thrown"); + } catch (IllegalStateException e) { + assertEquals("poller is closed!", e.getMessage()); + } + } +} diff --git a/server/src/test/java/org/opensearch/indices/pollingingest/MessageProcessorTests.java b/server/src/test/java/org/opensearch/indices/pollingingest/MessageProcessorTests.java new file mode 100644 index 0000000000000..273e25c0a5bfc --- /dev/null +++ b/server/src/test/java/org/opensearch/indices/pollingingest/MessageProcessorTests.java @@ -0,0 +1,99 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.indices.pollingingest; + +import org.opensearch.index.engine.Engine; +import org.opensearch.index.engine.FakeIngestionSource; +import org.opensearch.index.engine.IngestionEngine; +import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.DocumentMapperForType; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.index.mapper.ParsedDocument; +import org.opensearch.index.mapper.SourceToParse; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +import org.mockito.ArgumentCaptor; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class MessageProcessorTests extends OpenSearchTestCase { + private IngestionEngine ingestionEngine; + private DocumentMapper documentMapper; + private DocumentMapperForType documentMapperForType; + private MessageProcessorRunnable.MessageProcessor processor; + + @Before + public void setUp() throws Exception { + super.setUp(); + ingestionEngine = mock(IngestionEngine.class); + documentMapperForType = mock(DocumentMapperForType.class); + when(ingestionEngine.getDocumentMapperForType()).thenReturn(documentMapperForType); + + documentMapper = mock(DocumentMapper.class); + when(documentMapperForType.getDocumentMapper()).thenReturn(documentMapper); + processor = new MessageProcessorRunnable.MessageProcessor(ingestionEngine, "index"); + } + + public void testGetIndexOperation() throws IOException { + byte[] payload = "{\"_id\":\"1\",\"_source\":{\"name\":\"bob\", \"age\": 24}}".getBytes(StandardCharsets.UTF_8); + FakeIngestionSource.FakeIngestionShardPointer pointer = new FakeIngestionSource.FakeIngestionShardPointer(0); + + ParsedDocument parsedDocument = mock(ParsedDocument.class); + when(documentMapper.parse(any())).thenReturn(parsedDocument); + when(parsedDocument.rootDoc()).thenReturn(new ParseContext.Document()); + + Engine.Operation operation = processor.getOperation(payload, pointer); + + assertTrue(operation instanceof Engine.Index); + ArgumentCaptor captor = ArgumentCaptor.forClass(SourceToParse.class); + verify(documentMapper).parse(captor.capture()); + assertEquals("index", captor.getValue().index()); + assertEquals("1", captor.getValue().id()); + } + + public void testGetDeleteOperation() throws IOException { + byte[] payload = "{\"_id\":\"1\",\"_op_type\":\"delete\"}".getBytes(StandardCharsets.UTF_8); + FakeIngestionSource.FakeIngestionShardPointer pointer = new FakeIngestionSource.FakeIngestionShardPointer(0); + + Engine.Operation operation = processor.getOperation(payload, pointer); + + assertTrue(operation instanceof Engine.Delete); + Engine.Delete deleteOperation = (Engine.Delete) operation; + assertEquals("1", deleteOperation.id()); + } + + public void testSkipNoSourceIndexOperation() throws IOException { + byte[] payload = "{\"_id\":\"1\"}".getBytes(StandardCharsets.UTF_8); + FakeIngestionSource.FakeIngestionShardPointer pointer = new FakeIngestionSource.FakeIngestionShardPointer(0); + + Engine.Operation operation = processor.getOperation(payload, pointer); + assertNull(operation); + + // source has wrong type + payload = "{\"_id\":\"1\", \"_source\":1}".getBytes(StandardCharsets.UTF_8); + + operation = processor.getOperation(payload, pointer); + assertNull(operation); + } + + public void testUnsupportedOperation() throws IOException { + byte[] payload = "{\"_id\":\"1\", \"_op_tpe\":\"update\"}".getBytes(StandardCharsets.UTF_8); + FakeIngestionSource.FakeIngestionShardPointer pointer = new FakeIngestionSource.FakeIngestionShardPointer(0); + + Engine.Operation operation = processor.getOperation(payload, pointer); + assertNull(operation); + } +} diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 493a3ed431e00..a50c1d704362e 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2102,6 +2102,7 @@ public void onFailure(final Exception e) { repositoriesServiceReference::get, null, new RemoteStoreStatsTrackerFactory(clusterService, settings), + emptyMap(), DefaultRecoverySettings.INSTANCE, new CacheModule(new ArrayList<>(), settings).getCacheService(), DefaultRemoteStoreSettings.INSTANCE diff --git a/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java index 3403425d89254..d6cd5cfb81dc4 100644 --- a/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/engine/EngineTestCase.java @@ -96,6 +96,7 @@ import org.opensearch.index.codec.CodecService; import org.opensearch.index.fieldvisitor.IdOnlyFieldVisitor; import org.opensearch.index.mapper.DocumentMapper; +import org.opensearch.index.mapper.DocumentMapperForType; import org.opensearch.index.mapper.IdFieldMapper; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.Mapping; @@ -987,6 +988,40 @@ protected EngineConfig config( .build(); } + /** + * Override config with ingestion engine configs + */ + protected EngineConfig config(EngineConfig config, Supplier documentMapperForTypeSupplier) { + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings( + "test", + Settings.builder().put(config.getIndexSettings().getSettings()).build() + ); + return new EngineConfig.Builder().shardId(config.getShardId()) + .threadPool(config.getThreadPool()) + .indexSettings(indexSettings) + .warmer(config.getWarmer()) + .store(config.getStore()) + .mergePolicy(config.getMergePolicy()) + .analyzer(config.getAnalyzer()) + .similarity(config.getSimilarity()) + .codecService(new CodecService(null, indexSettings, logger)) + .eventListener(config.getEventListener()) + .queryCache(config.getQueryCache()) + .queryCachingPolicy(config.getQueryCachingPolicy()) + .translogConfig(config.getTranslogConfig()) + .flushMergesAfter(config.getFlushMergesAfter()) + .externalRefreshListener(config.getExternalRefreshListener()) + .internalRefreshListener(config.getInternalRefreshListener()) + .indexSort(config.getIndexSort()) + .circuitBreakerService(config.getCircuitBreakerService()) + .globalCheckpointSupplier(config.getGlobalCheckpointSupplier()) + .retentionLeasesSupplier(config.retentionLeasesSupplier()) + .primaryTermSupplier(config.getPrimaryTermSupplier()) + .tombstoneDocSupplier(config.getTombstoneDocSupplier()) + .documentMapperForTypeSupplier(documentMapperForTypeSupplier) + .build(); + } + protected EngineConfig noOpConfig(IndexSettings indexSettings, Store store, Path translogPath) { return noOpConfig(indexSettings, store, translogPath, null); } @@ -1541,6 +1576,10 @@ public static void assertAtMostOneLuceneDocumentPerSequenceNumber(IndexSettings } public static MapperService createMapperService() throws IOException { + return createMapperService("{\"properties\": {}}"); + } + + public static MapperService createMapperService(String mapping) throws IOException { IndexMetadata indexMetadata = IndexMetadata.builder("test") .settings( Settings.builder() @@ -1548,7 +1587,7 @@ public static MapperService createMapperService() throws IOException { .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) ) - .putMapping("{\"properties\": {}}") + .putMapping(mapping) .build(); MapperService mapperService = MapperTestUtils.newMapperService( new NamedXContentRegistry(ClusterModule.getNamedXWriteables()), diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index e1eafb8570022..318549f676edf 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -765,6 +765,13 @@ public final void createIndex(String name, Settings indexSettings) { assertAcked(prepareCreate(name).setSettings(indexSettings)); } + /** + * creates an index with the given setting and mapping + */ + public final void createIndex(String name, Settings indexSettings, String mapping) { + assertAcked(prepareCreate(name).setSettings(indexSettings).setMapping(mapping)); + } + /** * creates an index with the given setting */