From 009ea27a0f626abafa6da196ed76c7c3a2429f63 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Thu, 26 Oct 2023 20:56:17 -0700 Subject: [PATCH 01/22] Add initial impl with basic IT Signed-off-by: Chen Dai --- .../opensearch/flint/spark/FlintSpark.scala | 11 +- .../spark/FlintSparkIndexOperation.scala | 186 ++++++++++++++++++ .../FlintSparkIndexOperationITSuite.scala | 58 ++++++ 3 files changed, 254 insertions(+), 1 deletion(-) create mode 100644 flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala create mode 100644 integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 632730afd..bcd728e87 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -88,7 +88,16 @@ class FlintSpark(val spark: SparkSession) { } } else { val metadata = index.metadata() - flintClient.createIndex(indexName, metadata) + + FlintSparkIndexOperation(indexName, flintClient) + .before(latest => latest.state == "empty") + .transient(latest => latest.copy(state = "creating")) + .operation { + flintClient.createIndex(indexName, metadata) + null + } + .after(latest => latest.copy(state = "created")) + .execute() } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala new file mode 100644 index 000000000..06fed3e0e --- /dev/null +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala @@ -0,0 +1,186 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.spark + +import java.io.IOException +import java.util.Base64 + +import org.json4s.{Formats, NoTypeHints} +import org.json4s.native.JsonMethods.parse +import org.json4s.native.Serialization +import org.opensearch.action.get.GetRequest +import org.opensearch.action.index.IndexRequest +import org.opensearch.action.support.WriteRequest +import org.opensearch.action.update.UpdateRequest +import org.opensearch.client.RequestOptions +import org.opensearch.common.xcontent.XContentType +import org.opensearch.flint.core.FlintClient +import org.opensearch.flint.spark.FlintSparkIndexOperation.FlintMetadataLogEntry + +case class FlintSparkIndexOperation[T](flintIndexName: String, flintClient: FlintClient) { + + /** Reuse query request index as Flint metadata log store */ + private val metadataLogIndexName = ".query_request_history_mys3" // + ds name + + /** No need to query Flint index metadata */ + private val latestId = Base64.getEncoder.encodeToString(flintIndexName.getBytes) + + private var beforeCondition: FlintMetadataLogEntry => Boolean = null + private var transientAction: FlintMetadataLogEntry => FlintMetadataLogEntry = null + private var opAction: () => T = null + private var afterAction: FlintMetadataLogEntry => FlintMetadataLogEntry = null + + def before(condition: FlintMetadataLogEntry => Boolean): FlintSparkIndexOperation[T] = { + this.beforeCondition = condition + this + } + + def transient( + action: FlintMetadataLogEntry => FlintMetadataLogEntry): FlintSparkIndexOperation[T] = { + this.transientAction = action + this + } + + def operation(action: () => T): FlintSparkIndexOperation[T] = { + this.opAction = action + this + } + + def after( + action: FlintMetadataLogEntry => FlintMetadataLogEntry): FlintSparkIndexOperation[T] = { + this.afterAction = action + this + } + + def execute(): T = { + var latest = getLatestLogEntry + if (beforeCondition(latest)) { + updateDoc(transientAction(latest)) + val result = opAction() + updateDoc(afterAction(getLatestLogEntry)) + result + } else { + throw new IllegalStateException() + } + } + + private def getLatestLogEntry: FlintMetadataLogEntry = { + val latest = getDoc(latestId) + latest.getOrElse(FlintMetadataLogEntry()) + } + + // Visible for IT + def getDoc(docId: String): Option[FlintMetadataLogEntry] = { + val client = flintClient.createClient() + try { + val response = + client.get(new GetRequest(metadataLogIndexName, docId), RequestOptions.DEFAULT) + Some( + new FlintMetadataLogEntry( + response.getId, + response.getSeqNo, + response.getPrimaryTerm, + response.getSourceAsMap)) + } catch { + case _: Exception => None + } finally { + client.close() + } + } + + private def createDoc(logEntry: FlintMetadataLogEntry): Unit = { + val client = flintClient.createClient() + try { + client.index( + new IndexRequest() + .index(metadataLogIndexName) + .id(logEntry.docId) + .source(logEntry.toJson, XContentType.JSON), + RequestOptions.DEFAULT) + } finally { + client.close() + } + } + + private def updateDoc(logEntry: FlintMetadataLogEntry): Unit = { + if (logEntry.docId.isEmpty) { + createDoc(logEntry.copy(docId = latestId)) + } else { + updateIf(logEntry.docId, logEntry.toJson, logEntry.seqNo, logEntry.primaryTerm) + } + } + + def updateIf(id: String, doc: String, seqNo: Long, primaryTerm: Long): Unit = { + try { + val client = flintClient.createClient + try { + val updateRequest = new UpdateRequest(metadataLogIndexName, id) + .doc(doc, XContentType.JSON) + .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(seqNo) + .setIfPrimaryTerm(primaryTerm) + client.update(updateRequest, RequestOptions.DEFAULT) + } catch { + case e: IOException => + throw new RuntimeException( + String.format( + "Failed to execute update request on index: %s, id: %s", + metadataLogIndexName, + id), + e) + } finally if (client != null) client.close() + } + } +} + +object FlintSparkIndexOperation { + + case class FlintMetadataLogEntry( + val docId: String = "", + val seqNo: Long = -1, + val primaryTerm: Long = -1, + val state: String = "empty", + val dataSource: String = "", // TODO: get from Spark conf + val error: String = "") { + + def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { + this( + docId, + seqNo, + primaryTerm, + map.get("state").asInstanceOf[String], + map.get("dataSourceName").asInstanceOf[String], + map.get("error").asInstanceOf[String]) + } + + def toJson: String = { + s""" + |{ + | "version": "1.0", + | "type": "flintindexstate", + | "state": "$state", + | "applicationId": "${sys.env.getOrElse( + "SERVERLESS_EMR_VIRTUAL_CLUSTER_ID", + "unknown")}", + | "jobId": "${sys.env.getOrElse("SERVERLESS_EMR_JOB_ID", "unknown")}", + | "dataSourceName": "$dataSource", + | "lastUpdateTime": "${System.currentTimeMillis()}", + | "error": "$error" + |} + |""".stripMargin + } + } + + def fromJson(json: String): Unit = { + implicit val formats: Formats = Serialization.formats(NoTypeHints) + + val meta = parse(json) + val state = (meta \ "state").extract[String] + + val entry = new FlintMetadataLogEntry + entry.copy(state = state) + } +} diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala new file mode 100644 index 000000000..83e4c1756 --- /dev/null +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.spark + +import java.util.{Base64, Collections} + +import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} + +import org.opensearch.action.get.GetRequest +import org.opensearch.client.RequestOptions +import org.opensearch.client.indices.CreateIndexRequest +import org.opensearch.flint.core.FlintOptions +import org.opensearch.flint.core.storage.FlintOpenSearchClient +import org.scalatest.matchers.should.Matchers + +class FlintSparkIndexOperationITSuite extends FlintSparkSuite with Matchers { + + val testFlintIndex = "flint_test_index" + val testMetadataLogIndex = ".query_request_history_mys3" + val testLatestId = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) + + override def beforeAll(): Unit = { + super.beforeAll() + openSearchClient + .indices() + .create(new CreateIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + } + + test("test") { + val flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) + + FlintSparkIndexOperation(testFlintIndex, flintClient) + .before(latest => { + // latest.docId shouldBe + latest.state shouldBe "empty" + true + }) + .transient(latest => latest.copy(state = "creating")) + .operation(() => { + latestLogEntry should contain("state" -> "creating") + null + }) + .after(latest => latest.copy(state = "created")) + .execute() + + latestLogEntry should contain("state" -> "created") + } + + private def latestLogEntry: Map[String, AnyRef] = { + val response = openSearchClient + .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) + + Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap + } +} From c6fd5e2b26f5788aabe4968e7243ecc21291d9b7 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Fri, 27 Oct 2023 08:45:52 -0700 Subject: [PATCH 02/22] Move to Flint core Signed-off-by: Chen Dai --- .../opensearch/flint/core/FlintClient.java | 9 ++ .../metadata/log/FlintMetadataLogEntry.scala | 40 +++++ .../metadata/log/OptimisticTransaction.java | 21 +++ .../core/storage/FlintOpenSearchClient.java | 5 + .../OpenSearchOptimisticTransaction.java | 139 ++++++++++++++++++ .../FlintSparkIndexOperationITSuite.scala | 13 +- 6 files changed, 220 insertions(+), 7 deletions(-) create mode 100644 flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala create mode 100644 flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java create mode 100644 flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java index 9519df8bc..9706525b7 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java @@ -9,6 +9,7 @@ import org.opensearch.client.RestHighLevelClient; import org.opensearch.flint.core.metadata.FlintMetadata; +import org.opensearch.flint.core.metadata.log.OptimisticTransaction; import org.opensearch.flint.core.storage.FlintReader; import org.opensearch.flint.core.storage.FlintWriter; @@ -18,6 +19,14 @@ */ public interface FlintClient { + /** + * Start a new optimistic transaction. + * + * @param indexName index name + * @return transaction handle + */ + OptimisticTransaction startTransaction(String indexName); + /** * Create a Flint index with the metadata given. * diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala new file mode 100644 index 000000000..af218d564 --- /dev/null +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.core.metadata.log + +case class FlintMetadataLogEntry( + docId: String = "", + seqNo: Long = -1, + primaryTerm: Long = -1, + state: String = "empty", + dataSource: String = "", // TODO: get from Spark conf + error: String = "") { + + def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { + this( + docId, + seqNo, + primaryTerm, + map.get("state").asInstanceOf[String], + map.get("dataSourceName").asInstanceOf[String], + map.get("error").asInstanceOf[String]) + } + + def toJson: String = { + s""" + |{ + | "version": "1.0", + | "type": "flintindexstate", + | "state": "$state", + | "applicationId": "${sys.env.getOrElse("SERVERLESS_EMR_VIRTUAL_CLUSTER_ID", "unknown")}", + | "jobId": "${sys.env.getOrElse("SERVERLESS_EMR_JOB_ID", "unknown")}", + | "dataSourceName": "$dataSource", + | "lastUpdateTime": "${System.currentTimeMillis()}", + | "error": "$error" + |} + |""".stripMargin + } +} diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java new file mode 100644 index 000000000..a97f0de65 --- /dev/null +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.core.metadata.log; + +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; + +public interface OptimisticTransaction { + + OptimisticTransaction initialLog(Predicate initialCondition); + + OptimisticTransaction transientLog(Function action); + + OptimisticTransaction finalLog(Function action); + + T execute(Supplier action); +} diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index ff2761856..19ee6e9e5 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -42,6 +42,7 @@ import org.opensearch.flint.core.FlintOptions; import org.opensearch.flint.core.auth.AWSRequestSigningApacheInterceptor; import org.opensearch.flint.core.metadata.FlintMetadata; +import org.opensearch.flint.core.metadata.log.OptimisticTransaction; import org.opensearch.index.query.AbstractQueryBuilder; import org.opensearch.index.query.MatchAllQueryBuilder; import org.opensearch.index.query.QueryBuilder; @@ -68,6 +69,10 @@ public FlintOpenSearchClient(FlintOptions options) { this.options = options; } + @Override public OptimisticTransaction startTransaction(String indexName) { + return new OpenSearchOptimisticTransaction<>(indexName, this); + } + @Override public void createIndex(String indexName, FlintMetadata metadata) { String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java new file mode 100644 index 000000000..c62507b50 --- /dev/null +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -0,0 +1,139 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.core.storage; + +import java.io.IOException; +import java.util.Base64; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.function.Supplier; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.action.update.UpdateResponse; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.flint.core.FlintClient; +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry; +import org.opensearch.flint.core.metadata.log.OptimisticTransaction; + +public class OpenSearchOptimisticTransaction implements OptimisticTransaction { + + private final FlintClient flintClient; + + // Reuse query request index as Flint metadata log store + private final String metadataLogIndexName = ".query_request_history_mys3"; // + ds name + + // No need to query Flint index metadata + private final String latestId; + + private Predicate initialCondition = null; + private Function transientAction = null; + private Function finalAction = null; + + public OpenSearchOptimisticTransaction(String flintIndexName, FlintClient flintClient) { + this.latestId = Base64.getEncoder().encodeToString(flintIndexName.getBytes()); + this.flintClient = flintClient; + } + + @Override + public OpenSearchOptimisticTransaction initialLog(Predicate initialCondition) { + this.initialCondition = initialCondition; + return this; + } + + @Override + public OpenSearchOptimisticTransaction transientLog(Function action) { + this.transientAction = action; + return this; + } + + @Override + public OpenSearchOptimisticTransaction finalLog(Function action) { + this.finalAction = action; + return this; + } + + @Override + public T execute(Supplier action) { + Objects.requireNonNull(initialCondition); + Objects.requireNonNull(transientAction); + Objects.requireNonNull(finalAction); + + FlintMetadataLogEntry latest = getLatestLogEntry(); + if (initialCondition.test(latest)) { + updateDoc(transientAction.apply(latest)); + T result = action.get(); + updateDoc(finalAction.apply(getLatestLogEntry())); + return result; + } else { + throw new IllegalStateException(); + } + } + + private FlintMetadataLogEntry getLatestLogEntry() { + return getDoc(latestId).orElse( + new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", "")); + } + + // Visible for IT + public Optional getDoc(String docId) { + RestHighLevelClient client = flintClient.createClient(); + try { + GetResponse response = client.get(new GetRequest(metadataLogIndexName, docId), RequestOptions.DEFAULT); + return Optional.of(new FlintMetadataLogEntry(response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getSourceAsMap())); + } catch (Exception e) { + return Optional.empty(); + } + } + + private void createDoc(FlintMetadataLogEntry logEntry) { + try (RestHighLevelClient client = flintClient.createClient()) { + IndexRequest request = new IndexRequest() + .index(metadataLogIndexName) + .id(logEntry.docId()) + .source(logEntry.toJson(), XContentType.JSON); + client.index(request, RequestOptions.DEFAULT); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void updateDoc(FlintMetadataLogEntry logEntry) { + if (logEntry.docId().isEmpty()) { + createDoc( + logEntry.copy( + latestId, + logEntry.seqNo(), + logEntry.primaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error())); + } else { + updateIf(logEntry.docId(), logEntry.toJson(), logEntry.seqNo(), logEntry.primaryTerm()); + } + } + + public void updateIf(String id, String doc, long seqNo, long primaryTerm) { + try (RestHighLevelClient client = flintClient.createClient()) { + UpdateRequest updateRequest = new UpdateRequest(metadataLogIndexName, id) + .doc(doc, XContentType.JSON) + .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(seqNo) + .setIfPrimaryTerm(primaryTerm); + UpdateResponse response = client.update(updateRequest, RequestOptions.DEFAULT); + } catch (IOException e) { + throw new RuntimeException( + String.format("Failed to execute update request on index: %s, id: %s", metadataLogIndexName, id), + e); + } + } +} diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala index 83e4c1756..b22e565f6 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala @@ -32,19 +32,18 @@ class FlintSparkIndexOperationITSuite extends FlintSparkSuite with Matchers { test("test") { val flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) - FlintSparkIndexOperation(testFlintIndex, flintClient) - .before(latest => { - // latest.docId shouldBe + flintClient + .startTransaction(testFlintIndex) + .initialLog(latest => { latest.state shouldBe "empty" true }) - .transient(latest => latest.copy(state = "creating")) - .operation(() => { + .transientLog(latest => latest.copy(state = "creating")) + .finalLog(latest => latest.copy(state = "created")) + .execute(() => { latestLogEntry should contain("state" -> "creating") null }) - .after(latest => latest.copy(state = "created")) - .execute() latestLogEntry should contain("state" -> "created") } From 0fb0f2a448231c7293a42cc540fc9b79e98c09ef Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Fri, 27 Oct 2023 08:50:12 -0700 Subject: [PATCH 03/22] Switch to new transaction API Signed-off-by: Chen Dai --- .../opensearch/flint/spark/FlintSpark.scala | 14 +- .../spark/FlintSparkIndexOperation.scala | 186 ------------------ .../FlintOpenSearchTransactionITSuite.scala} | 6 +- 3 files changed, 10 insertions(+), 196 deletions(-) delete mode 100644 flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala rename integ-test/src/test/scala/org/opensearch/flint/{spark/FlintSparkIndexOperationITSuite.scala => core/FlintOpenSearchTransactionITSuite.scala} (90%) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index bcd728e87..2430666da 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -89,15 +89,15 @@ class FlintSpark(val spark: SparkSession) { } else { val metadata = index.metadata() - FlintSparkIndexOperation(indexName, flintClient) - .before(latest => latest.state == "empty") - .transient(latest => latest.copy(state = "creating")) - .operation { + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == "empty") + .transientLog(latest => latest.copy(state = "creating")) + .finalLog(latest => latest.copy(state = "created")) + .execute(() => { flintClient.createIndex(indexName, metadata) null - } - .after(latest => latest.copy(state = "created")) - .execute() + }) } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala deleted file mode 100644 index 06fed3e0e..000000000 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSparkIndexOperation.scala +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.flint.spark - -import java.io.IOException -import java.util.Base64 - -import org.json4s.{Formats, NoTypeHints} -import org.json4s.native.JsonMethods.parse -import org.json4s.native.Serialization -import org.opensearch.action.get.GetRequest -import org.opensearch.action.index.IndexRequest -import org.opensearch.action.support.WriteRequest -import org.opensearch.action.update.UpdateRequest -import org.opensearch.client.RequestOptions -import org.opensearch.common.xcontent.XContentType -import org.opensearch.flint.core.FlintClient -import org.opensearch.flint.spark.FlintSparkIndexOperation.FlintMetadataLogEntry - -case class FlintSparkIndexOperation[T](flintIndexName: String, flintClient: FlintClient) { - - /** Reuse query request index as Flint metadata log store */ - private val metadataLogIndexName = ".query_request_history_mys3" // + ds name - - /** No need to query Flint index metadata */ - private val latestId = Base64.getEncoder.encodeToString(flintIndexName.getBytes) - - private var beforeCondition: FlintMetadataLogEntry => Boolean = null - private var transientAction: FlintMetadataLogEntry => FlintMetadataLogEntry = null - private var opAction: () => T = null - private var afterAction: FlintMetadataLogEntry => FlintMetadataLogEntry = null - - def before(condition: FlintMetadataLogEntry => Boolean): FlintSparkIndexOperation[T] = { - this.beforeCondition = condition - this - } - - def transient( - action: FlintMetadataLogEntry => FlintMetadataLogEntry): FlintSparkIndexOperation[T] = { - this.transientAction = action - this - } - - def operation(action: () => T): FlintSparkIndexOperation[T] = { - this.opAction = action - this - } - - def after( - action: FlintMetadataLogEntry => FlintMetadataLogEntry): FlintSparkIndexOperation[T] = { - this.afterAction = action - this - } - - def execute(): T = { - var latest = getLatestLogEntry - if (beforeCondition(latest)) { - updateDoc(transientAction(latest)) - val result = opAction() - updateDoc(afterAction(getLatestLogEntry)) - result - } else { - throw new IllegalStateException() - } - } - - private def getLatestLogEntry: FlintMetadataLogEntry = { - val latest = getDoc(latestId) - latest.getOrElse(FlintMetadataLogEntry()) - } - - // Visible for IT - def getDoc(docId: String): Option[FlintMetadataLogEntry] = { - val client = flintClient.createClient() - try { - val response = - client.get(new GetRequest(metadataLogIndexName, docId), RequestOptions.DEFAULT) - Some( - new FlintMetadataLogEntry( - response.getId, - response.getSeqNo, - response.getPrimaryTerm, - response.getSourceAsMap)) - } catch { - case _: Exception => None - } finally { - client.close() - } - } - - private def createDoc(logEntry: FlintMetadataLogEntry): Unit = { - val client = flintClient.createClient() - try { - client.index( - new IndexRequest() - .index(metadataLogIndexName) - .id(logEntry.docId) - .source(logEntry.toJson, XContentType.JSON), - RequestOptions.DEFAULT) - } finally { - client.close() - } - } - - private def updateDoc(logEntry: FlintMetadataLogEntry): Unit = { - if (logEntry.docId.isEmpty) { - createDoc(logEntry.copy(docId = latestId)) - } else { - updateIf(logEntry.docId, logEntry.toJson, logEntry.seqNo, logEntry.primaryTerm) - } - } - - def updateIf(id: String, doc: String, seqNo: Long, primaryTerm: Long): Unit = { - try { - val client = flintClient.createClient - try { - val updateRequest = new UpdateRequest(metadataLogIndexName, id) - .doc(doc, XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) - .setIfSeqNo(seqNo) - .setIfPrimaryTerm(primaryTerm) - client.update(updateRequest, RequestOptions.DEFAULT) - } catch { - case e: IOException => - throw new RuntimeException( - String.format( - "Failed to execute update request on index: %s, id: %s", - metadataLogIndexName, - id), - e) - } finally if (client != null) client.close() - } - } -} - -object FlintSparkIndexOperation { - - case class FlintMetadataLogEntry( - val docId: String = "", - val seqNo: Long = -1, - val primaryTerm: Long = -1, - val state: String = "empty", - val dataSource: String = "", // TODO: get from Spark conf - val error: String = "") { - - def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { - this( - docId, - seqNo, - primaryTerm, - map.get("state").asInstanceOf[String], - map.get("dataSourceName").asInstanceOf[String], - map.get("error").asInstanceOf[String]) - } - - def toJson: String = { - s""" - |{ - | "version": "1.0", - | "type": "flintindexstate", - | "state": "$state", - | "applicationId": "${sys.env.getOrElse( - "SERVERLESS_EMR_VIRTUAL_CLUSTER_ID", - "unknown")}", - | "jobId": "${sys.env.getOrElse("SERVERLESS_EMR_JOB_ID", "unknown")}", - | "dataSourceName": "$dataSource", - | "lastUpdateTime": "${System.currentTimeMillis()}", - | "error": "$error" - |} - |""".stripMargin - } - } - - def fromJson(json: String): Unit = { - implicit val formats: Formats = Serialization.formats(NoTypeHints) - - val meta = parse(json) - val state = (meta \ "state").extract[String] - - val entry = new FlintMetadataLogEntry - entry.copy(state = state) - } -} diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala similarity index 90% rename from integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala rename to integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index b22e565f6..ff5072dc1 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkIndexOperationITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.flint.spark +package org.opensearch.flint.core import java.util.{Base64, Collections} @@ -12,11 +12,11 @@ import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConv import org.opensearch.action.get.GetRequest import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest -import org.opensearch.flint.core.FlintOptions import org.opensearch.flint.core.storage.FlintOpenSearchClient +import org.opensearch.flint.spark.FlintSparkSuite import org.scalatest.matchers.should.Matchers -class FlintSparkIndexOperationITSuite extends FlintSparkSuite with Matchers { +class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { val testFlintIndex = "flint_test_index" val testMetadataLogIndex = ".query_request_history_mys3" From 7cf3323bc847a8e84b0674b48d5b02ad7e9c9ffb Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Fri, 27 Oct 2023 08:58:31 -0700 Subject: [PATCH 04/22] Add java doc Signed-off-by: Chen Dai --- .../metadata/log/OptimisticTransaction.java | 30 ++++++++++++++++++- .../OpenSearchOptimisticTransaction.java | 6 ++-- 2 files changed, 32 insertions(+), 4 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java index a97f0de65..a7a0bf7c9 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java @@ -9,13 +9,41 @@ import java.util.function.Predicate; import java.util.function.Supplier; +/** + * Optimistic transaction interface that represents a state transition on the state machine. + * In particular, this abstraction is trying to express: + * initial log (precondition) + * => transient log (with pending operation to do) + * => final log (after operation succeeds) + * For example, "empty" => creating (operation is to create index) => created + * + * @param result type + */ public interface OptimisticTransaction { + /** + * @param initialCondition initial precondition that the subsequent transition and action can proceed + * @return this transaction + */ OptimisticTransaction initialLog(Predicate initialCondition); + /** + * @param action action to generate transient log entry + * @return this transaction + */ OptimisticTransaction transientLog(Function action); + /** + * @param action action to generate final log entry + * @return this transaction + */ OptimisticTransaction finalLog(Function action); - T execute(Supplier action); + /** + * Execute the given operation with the given log transition above. + * + * @param operation operation + * @return result + */ + T execute(Supplier operation); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index c62507b50..3d9fb036c 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -30,7 +30,7 @@ public class OpenSearchOptimisticTransaction implements OptimisticTransaction private final FlintClient flintClient; // Reuse query request index as Flint metadata log store - private final String metadataLogIndexName = ".query_request_history_mys3"; // + ds name + private final String metadataLogIndexName = ".query_request_history_mys3"; // TODO: get suffix ds name from Spark conf // No need to query Flint index metadata private final String latestId; @@ -63,7 +63,7 @@ public OpenSearchOptimisticTransaction finalLog(Function action) { + public T execute(Supplier operation) { Objects.requireNonNull(initialCondition); Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); @@ -71,7 +71,7 @@ public T execute(Supplier action) { FlintMetadataLogEntry latest = getLatestLogEntry(); if (initialCondition.test(latest)) { updateDoc(transientAction.apply(latest)); - T result = action.get(); + T result = operation.get(); updateDoc(finalAction.apply(getLatestLogEntry())); return result; } else { From 77ab1d86034ed530a2d996f67d30eb3546529321 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Fri, 27 Oct 2023 09:21:47 -0700 Subject: [PATCH 05/22] Refactor OS query method Signed-off-by: Chen Dai --- .../metadata/log/FlintMetadataLogEntry.scala | 29 ++++- .../OpenSearchOptimisticTransaction.java | 111 +++++++++--------- 2 files changed, 78 insertions(+), 62 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index af218d564..2cd32dc78 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -5,13 +5,30 @@ package org.opensearch.flint.core.metadata.log +/** + * Flint metadata log entry. This is temporary and will merge field in FlintMetadata here and move + * implementation specific field, such as seqNo, primaryTerm, dataSource to properties. + * + * @param id + * log entry id + * @param seqNo + * OpenSearch sequence number + * @param primaryTerm + * OpenSearch primary term + * @param state + * Flint index state + * @param dataSource + * OpenSearch data source associated //TODO: remove? + * @param error + * error details if in error state + */ case class FlintMetadataLogEntry( - docId: String = "", - seqNo: Long = -1, - primaryTerm: Long = -1, - state: String = "empty", - dataSource: String = "", // TODO: get from Spark conf - error: String = "") { + var id: String = "", + var seqNo: Long = -1, + var primaryTerm: Long = -1, + var state: String = "empty", + var dataSource: String = "", // TODO: get from Spark conf + var error: String = "") { def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { this( diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index 3d9fb036c..a2b243d20 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -5,19 +5,17 @@ package org.opensearch.flint.core.storage; -import java.io.IOException; import java.util.Base64; import java.util.Objects; -import java.util.Optional; import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; +import org.opensearch.action.DocWriteResponse; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.WriteRequest; import org.opensearch.action.update.UpdateRequest; -import org.opensearch.action.update.UpdateResponse; import org.opensearch.client.RequestOptions; import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.xcontent.XContentType; @@ -25,14 +23,27 @@ import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry; import org.opensearch.flint.core.metadata.log.OptimisticTransaction; +/** + * Optimistic transaction implementation by OpenSearch OCC. + * For now use single doc instead of maintaining history of metadata log. + * + * @param result type + */ public class OpenSearchOptimisticTransaction implements OptimisticTransaction { + /** + * Flint client to create Rest OpenSearch client (This will be refactored later) + */ private final FlintClient flintClient; - // Reuse query request index as Flint metadata log store + /** + * Reuse query request index as Flint metadata log store + */ private final String metadataLogIndexName = ".query_request_history_mys3"; // TODO: get suffix ds name from Spark conf - // No need to query Flint index metadata + /** + * Doc id for latest log entry (Naming rule is static so no need to query Flint index metadata) + */ private final String latestId; private Predicate initialCondition = null; @@ -70,9 +81,11 @@ public T execute(Supplier operation) { FlintMetadataLogEntry latest = getLatestLogEntry(); if (initialCondition.test(latest)) { - updateDoc(transientAction.apply(latest)); + // TODO: log entry can be same? + createOrUpdateDoc(transientAction.apply(latest)); T result = operation.get(); - updateDoc(finalAction.apply(getLatestLogEntry())); + // TODO: don't get latest, use previous entry again. (set seqno in update) + createOrUpdateDoc(finalAction.apply(getLatestLogEntry())); return result; } else { throw new IllegalStateException(); @@ -80,60 +93,46 @@ public T execute(Supplier operation) { } private FlintMetadataLogEntry getLatestLogEntry() { - return getDoc(latestId).orElse( - new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", "")); - } - - // Visible for IT - public Optional getDoc(String docId) { - RestHighLevelClient client = flintClient.createClient(); - try { - GetResponse response = client.get(new GetRequest(metadataLogIndexName, docId), RequestOptions.DEFAULT); - return Optional.of(new FlintMetadataLogEntry(response.getId(), response.getSeqNo(), response.getPrimaryTerm(), response.getSourceAsMap())); - } catch (Exception e) { - return Optional.empty(); - } - } - - private void createDoc(FlintMetadataLogEntry logEntry) { try (RestHighLevelClient client = flintClient.createClient()) { - IndexRequest request = new IndexRequest() - .index(metadataLogIndexName) - .id(logEntry.docId()) - .source(logEntry.toJson(), XContentType.JSON); - client.index(request, RequestOptions.DEFAULT); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private void updateDoc(FlintMetadataLogEntry logEntry) { - if (logEntry.docId().isEmpty()) { - createDoc( - logEntry.copy( - latestId, - logEntry.seqNo(), - logEntry.primaryTerm(), - logEntry.state(), - logEntry.dataSource(), - logEntry.error())); - } else { - updateIf(logEntry.docId(), logEntry.toJson(), logEntry.seqNo(), logEntry.primaryTerm()); + GetResponse response = + client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); + return new FlintMetadataLogEntry( + response.getId(), + response.getSeqNo(), + response.getPrimaryTerm(), + response.getSourceAsMap()); + } catch (Exception e) { + return new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", ""); } } - public void updateIf(String id, String doc, long seqNo, long primaryTerm) { + private void createOrUpdateDoc(FlintMetadataLogEntry logEntry) { try (RestHighLevelClient client = flintClient.createClient()) { - UpdateRequest updateRequest = new UpdateRequest(metadataLogIndexName, id) - .doc(doc, XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) - .setIfSeqNo(seqNo) - .setIfPrimaryTerm(primaryTerm); - UpdateResponse response = client.update(updateRequest, RequestOptions.DEFAULT); - } catch (IOException e) { - throw new RuntimeException( - String.format("Failed to execute update request on index: %s, id: %s", metadataLogIndexName, id), - e); + DocWriteResponse response; + if (logEntry.id().isEmpty()) { + logEntry.id_$eq(latestId); + response = client.index( + new IndexRequest() + .index(metadataLogIndexName) + .id(logEntry.id()) + .source(logEntry.toJson(), XContentType.JSON), + RequestOptions.DEFAULT); + } else { + response = + client.update( + new UpdateRequest(metadataLogIndexName, logEntry.id()) + .doc(logEntry.toJson(), XContentType.JSON) + .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(logEntry.seqNo()) + .setIfPrimaryTerm(logEntry.primaryTerm()), + RequestOptions.DEFAULT); + } + + // Update seqNo and primaryTerm in log entry object + logEntry.seqNo_$eq(response.getSeqNo()); // TODO: convert log entry to Java class? + logEntry.primaryTerm_$eq(response.getPrimaryTerm()); + } catch (Exception e) { + throw new RuntimeException(e); // TODO: handle expression properly } } } From 0bc782fd8aa3c5c09089867c368a3a0e91a1413a Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Fri, 27 Oct 2023 09:36:43 -0700 Subject: [PATCH 06/22] Add more IT for exceptional case Signed-off-by: Chen Dai --- .../FlintOpenSearchTransactionITSuite.scala | 29 +++++++++++++++++-- 1 file changed, 26 insertions(+), 3 deletions(-) diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index ff5072dc1..731346aa4 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -9,6 +9,7 @@ import java.util.{Base64, Collections} import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest import org.opensearch.action.get.GetRequest import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest @@ -20,18 +21,29 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { val testFlintIndex = "flint_test_index" val testMetadataLogIndex = ".query_request_history_mys3" - val testLatestId = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) + val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) + var flintClient: FlintClient = _ override def beforeAll(): Unit = { super.beforeAll() + flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) + } + + override def beforeEach(): Unit = { + super.beforeEach() openSearchClient .indices() .create(new CreateIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) } - test("test") { - val flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) + override def afterEach(): Unit = { + openSearchClient + .indices() + .delete(new DeleteIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + super.afterEach() + } + test("normal transition from initial to transient to final log") { flintClient .startTransaction(testFlintIndex) .initialLog(latest => { @@ -48,6 +60,17 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { latestLogEntry should contain("state" -> "created") } + test("should exit if initial log entry doesn't meet precondition") { + the[IllegalStateException] thrownBy { + flintClient + .startTransaction(testFlintIndex) + .initialLog(_ => false) + .transientLog(latest => latest) + .finalLog(latest => latest) + .execute(() => {}) + } + } + private def latestLogEntry: Map[String, AnyRef] = { val response = openSearchClient .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) From 51689b141bc8f46a746664fe4e55c9d443803bdf Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 08:22:09 -0700 Subject: [PATCH 07/22] Add no transaction class to avoid impact on existing IT Signed-off-by: Chen Dai --- .../metadata/log/OptimisticTransaction.java | 27 +++++++- .../core/storage/FlintOpenSearchClient.java | 12 +++- .../OpenSearchOptimisticTransaction.java | 42 +++++++------ .../core/FlintOpenSearchClientSuite.scala | 6 ++ .../FlintOpenSearchTransactionITSuite.scala | 62 ++++++++++++++++++- 5 files changed, 126 insertions(+), 23 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java index a7a0bf7c9..e7bd84b22 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java @@ -15,7 +15,7 @@ * initial log (precondition) * => transient log (with pending operation to do) * => final log (after operation succeeds) - * For example, "empty" => creating (operation is to create index) => created + * For example, "empty" => creating (operation is to create index) => active * * @param result type */ @@ -46,4 +46,29 @@ public interface OptimisticTransaction { * @return result */ T execute(Supplier operation); + + /** + * No optimistic transaction. + */ + class NoOptimisticTransaction implements OptimisticTransaction { + @Override + public OptimisticTransaction initialLog(Predicate initialCondition) { + return this; + } + + @Override + public OptimisticTransaction transientLog(Function action) { + return this; + } + + @Override + public OptimisticTransaction finalLog(Function action) { + return this; + } + + @Override + public T execute(Supplier operation) { + return operation.get(); + } + }; } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 19ee6e9e5..026799d88 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -43,6 +43,7 @@ import org.opensearch.flint.core.auth.AWSRequestSigningApacheInterceptor; import org.opensearch.flint.core.metadata.FlintMetadata; import org.opensearch.flint.core.metadata.log.OptimisticTransaction; +import org.opensearch.flint.core.metadata.log.OptimisticTransaction.NoOptimisticTransaction; import org.opensearch.index.query.AbstractQueryBuilder; import org.opensearch.index.query.MatchAllQueryBuilder; import org.opensearch.index.query.QueryBuilder; @@ -70,7 +71,16 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public OptimisticTransaction startTransaction(String indexName) { - return new OpenSearchOptimisticTransaction<>(indexName, this); + String metaLogIndexName = ".query_request_history_mys3"; + try (RestHighLevelClient client = createClient()) { + if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { + return new OpenSearchOptimisticTransaction<>(this, indexName, metaLogIndexName); + } else { + return new NoOptimisticTransaction<>(); + } + } catch (IOException e) { + throw new IllegalStateException("Failed to check if index metadata log index exists " + metaLogIndexName, e); + } } @Override public void createIndex(String indexName, FlintMetadata metadata) { diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index a2b243d20..5970703b3 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -5,6 +5,7 @@ package org.opensearch.flint.core.storage; +import java.io.IOException; import java.util.Base64; import java.util.Objects; import java.util.function.Function; @@ -39,7 +40,7 @@ public class OpenSearchOptimisticTransaction implements OptimisticTransaction /** * Reuse query request index as Flint metadata log store */ - private final String metadataLogIndexName = ".query_request_history_mys3"; // TODO: get suffix ds name from Spark conf + private final String metadataLogIndexName; /** * Doc id for latest log entry (Naming rule is static so no need to query Flint index metadata) @@ -50,9 +51,10 @@ public class OpenSearchOptimisticTransaction implements OptimisticTransaction private Function transientAction = null; private Function finalAction = null; - public OpenSearchOptimisticTransaction(String flintIndexName, FlintClient flintClient) { - this.latestId = Base64.getEncoder().encodeToString(flintIndexName.getBytes()); + public OpenSearchOptimisticTransaction(FlintClient flintClient, String flintIndexName, String metadataLogIndexName) { this.flintClient = flintClient; + this.latestId = Base64.getEncoder().encodeToString(flintIndexName.getBytes()); + this.metadataLogIndexName = metadataLogIndexName; } @Override @@ -79,16 +81,22 @@ public T execute(Supplier operation) { Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); - FlintMetadataLogEntry latest = getLatestLogEntry(); - if (initialCondition.test(latest)) { - // TODO: log entry can be same? - createOrUpdateDoc(transientAction.apply(latest)); - T result = operation.get(); - // TODO: don't get latest, use previous entry again. (set seqno in update) - createOrUpdateDoc(finalAction.apply(getLatestLogEntry())); - return result; - } else { - throw new IllegalStateException(); + try { + FlintMetadataLogEntry latest = getLatestLogEntry(); + if (initialCondition.test(latest)) { + // TODO: log entry can be same? + createOrUpdateDoc(transientAction.apply(latest)); + + T result = operation.get(); + + // TODO: don't get latest, use previous entry again. (set seqno in update) + createOrUpdateDoc(finalAction.apply(getLatestLogEntry())); + return result; + } else { + throw new IllegalStateException(); + } + } catch (IOException e) { + throw new RuntimeException(); } } @@ -101,15 +109,15 @@ private FlintMetadataLogEntry getLatestLogEntry() { response.getSeqNo(), response.getPrimaryTerm(), response.getSourceAsMap()); - } catch (Exception e) { + } catch (Exception e) { // TODO: resource not found exception? return new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", ""); } } - private void createOrUpdateDoc(FlintMetadataLogEntry logEntry) { + private void createOrUpdateDoc(FlintMetadataLogEntry logEntry) throws IOException { try (RestHighLevelClient client = flintClient.createClient()) { DocWriteResponse response; - if (logEntry.id().isEmpty()) { + if (logEntry.id().isEmpty()) { // TODO: Only create before initialLog for the first time logEntry.id_$eq(latestId); response = client.index( new IndexRequest() @@ -131,8 +139,6 @@ private void createOrUpdateDoc(FlintMetadataLogEntry logEntry) { // Update seqNo and primaryTerm in log entry object logEntry.seqNo_$eq(response.getSeqNo()); // TODO: convert log entry to Java class? logEntry.primaryTerm_$eq(response.getPrimaryTerm()); - } catch (Exception e) { - throw new RuntimeException(e); // TODO: handle expression properly } } } diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala index 5c799128c..e096723a4 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala @@ -16,6 +16,7 @@ import org.opensearch.client.opensearch.OpenSearchClient import org.opensearch.client.transport.rest_client.RestClientTransport import org.opensearch.flint.OpenSearchSuite import org.opensearch.flint.core.metadata.FlintMetadata +import org.opensearch.flint.core.metadata.log.OptimisticTransaction.NoOptimisticTransaction import org.opensearch.flint.core.storage.FlintOpenSearchClient import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -30,6 +31,11 @@ class FlintOpenSearchClientSuite extends AnyFlatSpec with OpenSearchSuite with M behavior of "Flint OpenSearch client" + it should "start no optimistic transaction if metadata log index doesn't exists" in { + val transaction = flintClient.startTransaction("test") + transaction shouldBe a[NoOptimisticTransaction[AnyRef]] + } + it should "create index successfully" in { val indexName = "test" val content = diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index 731346aa4..212545085 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -11,8 +11,12 @@ import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConv import org.opensearch.action.admin.indices.delete.DeleteIndexRequest import org.opensearch.action.get.GetRequest +import org.opensearch.action.index.IndexRequest +import org.opensearch.action.update.UpdateRequest import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest +import org.opensearch.common.xcontent.XContentType +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.storage.FlintOpenSearchClient import org.opensearch.flint.spark.FlintSparkSuite import org.scalatest.matchers.should.Matchers @@ -43,7 +47,7 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { super.afterEach() } - test("normal transition from initial to transient to final log") { + test("should transit from initial to final if initial is empty") { flintClient .startTransaction(testFlintIndex) .initialLog(latest => { @@ -51,13 +55,33 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { true }) .transientLog(latest => latest.copy(state = "creating")) - .finalLog(latest => latest.copy(state = "created")) + .finalLog(latest => latest.copy(state = "active")) .execute(() => { latestLogEntry should contain("state" -> "creating") null }) - latestLogEntry should contain("state" -> "created") + latestLogEntry should contain("state" -> "active") + } + + test("should transit from initial to final if initial is not empty but meet precondition") { + // Create doc first to simulate this scenario + createLatestLogEntry(FlintMetadataLogEntry(id = testLatestId, state = "active")) + + flintClient + .startTransaction(testFlintIndex) + .initialLog(latest => { + latest.state shouldBe "active" + true + }) + .transientLog(latest => latest.copy(state = "refreshing")) + .finalLog(latest => latest.copy(state = "active")) + .execute(() => { + latestLogEntry should contain("state" -> "refreshing") + null + }) + + latestLogEntry should contain("state" -> "active") } test("should exit if initial log entry doesn't meet precondition") { @@ -71,10 +95,42 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { } } + ignore("should exit if initial log entry updated by others when adding transient log entry") { + the[IllegalStateException] thrownBy { + flintClient + .startTransaction(testFlintIndex) + .initialLog(_ => true) + .transientLog(latest => { + // This update will happen first and thus cause version conflict + updateLatestLogEntry(latest, "deleting") + + latest.copy(state = "creating") + }) + .finalLog(latest => latest) + .execute(() => {}) + } + } + private def latestLogEntry: Map[String, AnyRef] = { val response = openSearchClient .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap } + + private def createLatestLogEntry(latest: FlintMetadataLogEntry): Unit = { + openSearchClient.index( + new IndexRequest() + .index(testMetadataLogIndex) + .id(testLatestId) + .source(latest.toJson, XContentType.JSON), + RequestOptions.DEFAULT) + } + + private def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: String): Unit = { + openSearchClient.update( + new UpdateRequest(testMetadataLogIndex, testLatestId) + .doc(latest.copy(state = newState).toJson, XContentType.JSON), + RequestOptions.DEFAULT) + } } From 78c0dc7581ea90ba80520d79be9be3956a37c20d Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 10:06:50 -0700 Subject: [PATCH 08/22] Make Flint metadata log entry case class and use copy Signed-off-by: Chen Dai --- .../metadata/log/FlintMetadataLogEntry.scala | 12 +- .../OpenSearchOptimisticTransaction.java | 108 +++++++++++------- 2 files changed, 70 insertions(+), 50 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index 2cd32dc78..5f0610d41 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -23,12 +23,12 @@ package org.opensearch.flint.core.metadata.log * error details if in error state */ case class FlintMetadataLogEntry( - var id: String = "", - var seqNo: Long = -1, - var primaryTerm: Long = -1, - var state: String = "empty", - var dataSource: String = "", // TODO: get from Spark conf - var error: String = "") { + id: String = "", + seqNo: Long = -1, + primaryTerm: Long = -1, + state: String = "empty", + dataSource: String = "", // TODO: get from Spark conf + error: String = "") { def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { this( diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index 5970703b3..e882779a8 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -11,12 +11,13 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; -import org.opensearch.action.DocWriteResponse; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.index.IndexResponse; import org.opensearch.action.support.WriteRequest; import org.opensearch.action.update.UpdateRequest; +import org.opensearch.action.update.UpdateResponse; import org.opensearch.client.RequestOptions; import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.xcontent.XContentType; @@ -81,64 +82,83 @@ public T execute(Supplier operation) { Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); - try { - FlintMetadataLogEntry latest = getLatestLogEntry(); - if (initialCondition.test(latest)) { - // TODO: log entry can be same? - createOrUpdateDoc(transientAction.apply(latest)); + FlintMetadataLogEntry latest = getLatestLogEntry(); + if (latest.id().isEmpty()) { + latest = createLogEntry(latest); + } - T result = operation.get(); + if (initialCondition.test(latest)) { + // TODO: log entry can be same? + latest = updateLogEntry(transientAction.apply(latest)); - // TODO: don't get latest, use previous entry again. (set seqno in update) - createOrUpdateDoc(finalAction.apply(getLatestLogEntry())); - return result; - } else { - throw new IllegalStateException(); - } - } catch (IOException e) { - throw new RuntimeException(); + T result = operation.get(); + + updateLogEntry(finalAction.apply(latest)); + return result; + } else { + throw new IllegalStateException(); } } private FlintMetadataLogEntry getLatestLogEntry() { - try (RestHighLevelClient client = flintClient.createClient()) { + RestHighLevelClient client = flintClient.createClient(); + try { GetResponse response = client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); - return new FlintMetadataLogEntry( - response.getId(), - response.getSeqNo(), - response.getPrimaryTerm(), - response.getSourceAsMap()); + + if (response.isExists()) { + return new FlintMetadataLogEntry( + response.getId(), + response.getSeqNo(), + response.getPrimaryTerm(), + response.getSourceAsMap()); + } else { + return new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", ""); + } } catch (Exception e) { // TODO: resource not found exception? - return new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", ""); + throw new IllegalStateException("Failed to fetch latest metadata log entry", e); } } - private void createOrUpdateDoc(FlintMetadataLogEntry logEntry) throws IOException { + private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { try (RestHighLevelClient client = flintClient.createClient()) { - DocWriteResponse response; - if (logEntry.id().isEmpty()) { // TODO: Only create before initialLog for the first time - logEntry.id_$eq(latestId); - response = client.index( - new IndexRequest() - .index(metadataLogIndexName) - .id(logEntry.id()) - .source(logEntry.toJson(), XContentType.JSON), - RequestOptions.DEFAULT); - } else { - response = - client.update( - new UpdateRequest(metadataLogIndexName, logEntry.id()) - .doc(logEntry.toJson(), XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) - .setIfSeqNo(logEntry.seqNo()) - .setIfPrimaryTerm(logEntry.primaryTerm()), - RequestOptions.DEFAULT); - } + logEntry = logEntry.copy( + latestId, + logEntry.seqNo(), logEntry.primaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); + + IndexResponse response = client.index( + new IndexRequest() + .index(metadataLogIndexName) + .id(logEntry.id()) + .source(logEntry.toJson(), XContentType.JSON), + RequestOptions.DEFAULT); // Update seqNo and primaryTerm in log entry object - logEntry.seqNo_$eq(response.getSeqNo()); // TODO: convert log entry to Java class? - logEntry.primaryTerm_$eq(response.getPrimaryTerm()); + return logEntry.copy( + logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), + logEntry.state(), logEntry.dataSource(), logEntry.error()); + } catch (IOException e) { + throw new IllegalStateException("Failed to create initial log entry", e); + } + } + + private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { + try (RestHighLevelClient client = flintClient.createClient()) { + UpdateResponse response = + client.update( + new UpdateRequest(metadataLogIndexName, logEntry.id()) + .doc(logEntry.toJson(), XContentType.JSON) + .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(logEntry.seqNo()) + .setIfPrimaryTerm(logEntry.primaryTerm()), + RequestOptions.DEFAULT); + + // Update seqNo and primaryTerm in log entry object + return logEntry.copy( + logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), + logEntry.state(), logEntry.dataSource(), logEntry.error()); + } catch (IOException e) { + throw new IllegalStateException("Failed to update log entry: " + logEntry, e); } } } From ab1f3eb62d1a59a05f38a082c91fcf312b055c7d Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 10:55:47 -0700 Subject: [PATCH 09/22] Add logging in FlintSpark, FlintClient and Transaction layer Signed-off-by: Chen Dai --- .../core/storage/FlintOpenSearchClient.java | 11 ++++ .../OpenSearchOptimisticTransaction.java | 14 ++++-- .../opensearch/flint/spark/FlintSpark.scala | 50 +++++++++++++++---- .../FlintOpenSearchTransactionITSuite.scala | 2 +- 4 files changed, 62 insertions(+), 15 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 026799d88..3a3f9abc4 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -18,6 +18,7 @@ import java.util.Locale; import java.util.Objects; import java.util.concurrent.atomic.AtomicReference; +import java.util.logging.Logger; import java.util.stream.Collectors; import org.apache.http.HttpHost; import org.apache.http.auth.AuthScope; @@ -56,6 +57,8 @@ */ public class FlintOpenSearchClient implements FlintClient { + private static final Logger LOG = Logger.getLogger(FlintOpenSearchClient.class.getName()); + /** * {@link NamedXContentRegistry} from {@link SearchModule} used for construct {@link QueryBuilder} from DSL query string. */ @@ -71,6 +74,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public OptimisticTransaction startTransaction(String indexName) { + LOG.info("Starting transaction on index " + indexName); String metaLogIndexName = ".query_request_history_mys3"; try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { @@ -84,6 +88,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public void createIndex(String indexName, FlintMetadata metadata) { + LOG.info("Creating Flint index " + indexName + " with metadata " + metadata); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { CreateIndexRequest request = new CreateIndexRequest(osIndexName); @@ -100,6 +105,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public boolean exists(String indexName) { + LOG.info("Checking if Flint index exists " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { return client.indices().exists(new GetIndexRequest(osIndexName), RequestOptions.DEFAULT); @@ -109,6 +115,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public List getAllIndexMetadata(String indexNamePattern) { + LOG.info("Fetching all Flint index metadata for pattern " + indexNamePattern); String osIndexNamePattern = toLowercase(indexNamePattern); try (RestHighLevelClient client = createClient()) { GetIndexRequest request = new GetIndexRequest(osIndexNamePattern); @@ -125,6 +132,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public FlintMetadata getIndexMetadata(String indexName) { + LOG.info("Fetching Flint index metadata for " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { GetIndexRequest request = new GetIndexRequest(osIndexName); @@ -139,6 +147,7 @@ public FlintOpenSearchClient(FlintOptions options) { } @Override public void deleteIndex(String indexName) { + LOG.info("Deleting Flint index " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { DeleteIndexRequest request = new DeleteIndexRequest(osIndexName); @@ -157,6 +166,7 @@ public FlintOpenSearchClient(FlintOptions options) { * @return {@link FlintReader}. */ @Override public FlintReader createReader(String indexName, String query) { + LOG.info("Creating Flint index reader for " + indexName + " with query " + query); try { QueryBuilder queryBuilder = new MatchAllQueryBuilder(); if (!Strings.isNullOrEmpty(query)) { @@ -175,6 +185,7 @@ public FlintOpenSearchClient(FlintOptions options) { } public FlintWriter createWriter(String indexName) { + LOG.info("Creating Flint index writer for " + indexName); return new OpenSearchWriter(createClient(), toLowercase(indexName), options.getRefreshPolicy()); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index e882779a8..bc1925636 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -11,6 +11,8 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.function.Supplier; +import java.util.logging.Logger; +import org.opensearch.OpenSearchException; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; @@ -33,6 +35,8 @@ */ public class OpenSearchOptimisticTransaction implements OptimisticTransaction { + private static final Logger LOG = Logger.getLogger(OpenSearchOptimisticTransaction.class.getName()); + /** * Flint client to create Rest OpenSearch client (This will be refactored later) */ @@ -96,10 +100,12 @@ public T execute(Supplier operation) { updateLogEntry(finalAction.apply(latest)); return result; } else { - throw new IllegalStateException(); + throw new IllegalStateException("Exit due to initial log precondition not satisfied"); } } + // TODO: Move all these to FlintLogEntry <- FlintOpenSearchLogEntry + private FlintMetadataLogEntry getLatestLogEntry() { RestHighLevelClient client = flintClient.createClient(); try { @@ -121,6 +127,7 @@ private FlintMetadataLogEntry getLatestLogEntry() { } private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { + LOG.info("Creating log entry " + logEntry); try (RestHighLevelClient client = flintClient.createClient()) { logEntry = logEntry.copy( latestId, @@ -137,12 +144,13 @@ private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { return logEntry.copy( logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); - } catch (IOException e) { + } catch (OpenSearchException | IOException e) { throw new IllegalStateException("Failed to create initial log entry", e); } } private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { + LOG.info("Updating log entry " + logEntry); try (RestHighLevelClient client = flintClient.createClient()) { UpdateResponse response = client.update( @@ -157,7 +165,7 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { return logEntry.copy( logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); - } catch (IOException e) { + } catch (OpenSearchException | IOException e) { throw new IllegalStateException("Failed to update log entry: " + logEntry, e); } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 2430666da..150fda71e 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -17,6 +17,7 @@ import org.opensearch.flint.spark.mv.FlintSparkMaterializedView import org.opensearch.flint.spark.skipping.FlintSparkSkippingIndex import org.opensearch.flint.spark.skipping.FlintSparkSkippingStrategy.SkippingKindSerializer +import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.SaveMode._ import org.apache.spark.sql.flint.FlintDataSourceV2.FLINT_DATASOURCE @@ -27,7 +28,7 @@ import org.apache.spark.sql.streaming.{DataStreamWriter, Trigger} /** * Flint Spark integration API entrypoint. */ -class FlintSpark(val spark: SparkSession) { +class FlintSpark(val spark: SparkSession) extends Logging { /** Flint spark configuration */ private val flintSparkConf: FlintSparkConf = @@ -81,6 +82,7 @@ class FlintSpark(val spark: SparkSession) { * Ignore existing index */ def createIndex(index: FlintSparkIndex, ignoreIfExists: Boolean = false): Unit = { + logInfo(s"Creating Flint index $index with ignoreIfExists $ignoreIfExists") val indexName = index.name() if (flintClient.exists(indexName)) { if (!ignoreIfExists) { @@ -88,16 +90,16 @@ class FlintSpark(val spark: SparkSession) { } } else { val metadata = index.metadata() - - flintClient - .startTransaction(indexName) - .initialLog(latest => latest.state == "empty") - .transientLog(latest => latest.copy(state = "creating")) - .finalLog(latest => latest.copy(state = "created")) - .execute(() => { - flintClient.createIndex(indexName, metadata) - null - }) + try { + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == "empty") + .transientLog(latest => latest.copy(state = "creating")) + .finalLog(latest => latest.copy(state = "created")) + .execute(() => flintClient.createIndex(indexName, metadata)) + } catch { + case e: Exception => logError("Failed to create Flint index", e) + } } } @@ -112,8 +114,30 @@ class FlintSpark(val spark: SparkSession) { * refreshing job ID (empty if batch job for now) */ def refreshIndex(indexName: String, mode: RefreshMode): Option[String] = { + logInfo(s"Refreshing Flint index $indexName with mode $mode") val index = describeIndex(indexName) .getOrElse(throw new IllegalStateException(s"Index $indexName doesn't exist")) + + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == "active") + .transientLog(latest => latest.copy(state = "refreshing")) + .finalLog(latest => { + if (mode == FULL) { + latest.copy(state = "active") + } else { + // TODO: scheduling regular update on heartbeat timestamp + latest + } + }) + .execute(() => doRefreshIndex(index, indexName, mode)) + } + + // TODO: move to separate class + def doRefreshIndex( + index: FlintSparkIndex, + indexName: String, + mode: RefreshMode): Option[String] = { val options = index.options val tableName = index.metadata().source @@ -176,6 +200,7 @@ class FlintSpark(val spark: SparkSession) { * Flint index list */ def describeIndexes(indexNamePattern: String): Seq[FlintSparkIndex] = { + logInfo(s"Describing indexes with pattern $indexNamePattern") if (flintClient.exists(indexNamePattern)) { flintClient .getAllIndexMetadata(indexNamePattern) @@ -195,6 +220,7 @@ class FlintSpark(val spark: SparkSession) { * Flint index */ def describeIndex(indexName: String): Option[FlintSparkIndex] = { + logInfo(s"Describing index name $indexName") if (flintClient.exists(indexName)) { val metadata = flintClient.getIndexMetadata(indexName) val index = FlintSparkIndexFactory.create(metadata) @@ -213,7 +239,9 @@ class FlintSpark(val spark: SparkSession) { * true if exist and deleted, otherwise false */ def deleteIndex(indexName: String): Boolean = { + logInfo(s"Deleting Flint index $indexName") if (flintClient.exists(indexName)) { + stopRefreshingJob(indexName) flintClient.deleteIndex(indexName) true diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index 212545085..e61e67b31 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -95,7 +95,7 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { } } - ignore("should exit if initial log entry updated by others when adding transient log entry") { + test("should exit if initial log entry updated by others when adding transient log entry") { the[IllegalStateException] thrownBy { flintClient .startTransaction(testFlintIndex) From 627d35a1def85e1a727bb23eb350ff8131641dc0 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 11:25:28 -0700 Subject: [PATCH 10/22] Add index state enum Signed-off-by: Chen Dai --- .../metadata/log/FlintMetadataLogEntry.scala | 41 +++++++++++++++---- .../OpenSearchOptimisticTransaction.java | 4 +- .../opensearch/flint/spark/FlintSpark.scala | 29 +++++++++---- .../FlintOpenSearchTransactionITSuite.scala | 27 +++++------- .../spark/FlintSparkTransactionITSuite.scala | 10 +++++ 5 files changed, 78 insertions(+), 33 deletions(-) create mode 100644 integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index 5f0610d41..8b632e4f8 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -5,6 +5,9 @@ package org.opensearch.flint.core.metadata.log +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.IndexState + /** * Flint metadata log entry. This is temporary and will merge field in FlintMetadata here and move * implementation specific field, such as seqNo, primaryTerm, dataSource to properties. @@ -23,19 +26,19 @@ package org.opensearch.flint.core.metadata.log * error details if in error state */ case class FlintMetadataLogEntry( - id: String = "", - seqNo: Long = -1, - primaryTerm: Long = -1, - state: String = "empty", - dataSource: String = "", // TODO: get from Spark conf - error: String = "") { + id: String, + seqNo: Long, + primaryTerm: Long, + state: IndexState, + dataSource: String, // TODO: get from Spark conf + error: String) { def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { this( docId, seqNo, primaryTerm, - map.get("state").asInstanceOf[String], + IndexState.from(map.get("state").asInstanceOf[String]), map.get("dataSourceName").asInstanceOf[String], map.get("error").asInstanceOf[String]) } @@ -55,3 +58,27 @@ case class FlintMetadataLogEntry( |""".stripMargin } } + +object FlintMetadataLogEntry { + + /** + * Flint index state enum. + */ + object IndexState extends Enumeration { + type IndexState = Value + val EMPTY: IndexState.Value = Value("empty") + val CREATING: IndexState.Value = Value("creating") + val ACTIVE: IndexState.Value = Value("active") + val REFRESHING: IndexState.Value = Value("refreshing") + val DELETING: IndexState.Value = Value("deleting") + val DELETED: IndexState.Value = Value("deleted") + val FAILED: IndexState.Value = Value("failed") + val UNKNOWN: IndexState.Value = Value("unknown") + + def from(s: String): IndexState.Value = { + IndexState.values + .find(_.toString.equalsIgnoreCase(s)) + .getOrElse(IndexState.UNKNOWN) + } + } +} diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index bc1925636..aaef3696a 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -5,6 +5,8 @@ package org.opensearch.flint.core.storage; +import static org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState$; + import java.io.IOException; import java.util.Base64; import java.util.Objects; @@ -119,7 +121,7 @@ private FlintMetadataLogEntry getLatestLogEntry() { response.getPrimaryTerm(), response.getSourceAsMap()); } else { - return new FlintMetadataLogEntry("", -1, -1, "empty", "mys3", ""); + return new FlintMetadataLogEntry("", -1, -1, IndexState$.MODULE$.EMPTY(), "mys3", ""); } } catch (Exception e) { // TODO: resource not found exception? throw new IllegalStateException("Failed to fetch latest metadata log entry", e); diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 150fda71e..4b11e8450 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -10,6 +10,7 @@ import scala.collection.JavaConverters._ import org.json4s.{Formats, NoTypeHints} import org.json4s.native.Serialization import org.opensearch.flint.core.{FlintClient, FlintClientBuilder} +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState._ import org.opensearch.flint.spark.FlintSpark.RefreshMode.{FULL, INCREMENTAL, RefreshMode} import org.opensearch.flint.spark.FlintSparkIndex.{ID_COLUMN, StreamingRefresh} import org.opensearch.flint.spark.covering.FlintSparkCoveringIndex @@ -93,9 +94,9 @@ class FlintSpark(val spark: SparkSession) extends Logging { try { flintClient .startTransaction(indexName) - .initialLog(latest => latest.state == "empty") - .transientLog(latest => latest.copy(state = "creating")) - .finalLog(latest => latest.copy(state = "created")) + .initialLog(latest => latest.state == EMPTY) + .transientLog(latest => latest.copy(state = CREATING)) + .finalLog(latest => latest.copy(state = ACTIVE)) .execute(() => flintClient.createIndex(indexName, metadata)) } catch { case e: Exception => logError("Failed to create Flint index", e) @@ -120,11 +121,11 @@ class FlintSpark(val spark: SparkSession) extends Logging { flintClient .startTransaction(indexName) - .initialLog(latest => latest.state == "active") - .transientLog(latest => latest.copy(state = "refreshing")) + .initialLog(latest => latest.state == ACTIVE) + .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => { if (mode == FULL) { - latest.copy(state = "active") + latest.copy(state = ACTIVE) } else { // TODO: scheduling regular update on heartbeat timestamp latest @@ -241,9 +242,16 @@ class FlintSpark(val spark: SparkSession) extends Logging { def deleteIndex(indexName: String): Boolean = { logInfo(s"Deleting Flint index $indexName") if (flintClient.exists(indexName)) { - - stopRefreshingJob(indexName) - flintClient.deleteIndex(indexName) + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) + .transientLog(latest => latest.copy(state = DELETING)) + .finalLog(latest => latest.copy(state = DELETED)) + .execute(() => { + // TODO: share same transaction for now + stopRefreshingJob(indexName) + flintClient.deleteIndex(indexName) + }) true } else { false @@ -266,9 +274,12 @@ class FlintSpark(val spark: SparkSession) extends Logging { spark.streams.active.exists(_.name == indexName) private def stopRefreshingJob(indexName: String): Unit = { + logInfo(s"Terminating refreshing job $indexName") val job = spark.streams.active.find(_.name == indexName) if (job.isDefined) { job.get.stop() + } else { + logWarning("Refreshing job not found") } } diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index e61e67b31..dc9b9bd50 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -17,6 +17,7 @@ import org.opensearch.client.RequestOptions import org.opensearch.client.indices.CreateIndexRequest import org.opensearch.common.xcontent.XContentType import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState._ import org.opensearch.flint.core.storage.FlintOpenSearchClient import org.opensearch.flint.spark.FlintSparkSuite import org.scalatest.matchers.should.Matchers @@ -54,12 +55,9 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { latest.state shouldBe "empty" true }) - .transientLog(latest => latest.copy(state = "creating")) - .finalLog(latest => latest.copy(state = "active")) - .execute(() => { - latestLogEntry should contain("state" -> "creating") - null - }) + .transientLog(latest => latest.copy(state = CREATING)) + .finalLog(latest => latest.copy(state = ACTIVE)) + .execute(() => latestLogEntry should contain("state" -> "creating")) latestLogEntry should contain("state" -> "active") } @@ -74,12 +72,9 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { latest.state shouldBe "active" true }) - .transientLog(latest => latest.copy(state = "refreshing")) - .finalLog(latest => latest.copy(state = "active")) - .execute(() => { - latestLogEntry should contain("state" -> "refreshing") - null - }) + .transientLog(latest => latest.copy(state = REFRESHING)) + .finalLog(latest => latest.copy(state = ACTIVE)) + .execute(() => latestLogEntry should contain("state" -> "refreshing")) latestLogEntry should contain("state" -> "active") } @@ -101,10 +96,10 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { .startTransaction(testFlintIndex) .initialLog(_ => true) .transientLog(latest => { - // This update will happen first and thus cause version conflict - updateLatestLogEntry(latest, "deleting") + // This update will happen first and thus cause version conflict as expected + updateLatestLogEntry(latest, DELETING) - latest.copy(state = "creating") + latest.copy(state = CREATING) }) .finalLog(latest => latest) .execute(() => {}) @@ -127,7 +122,7 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { RequestOptions.DEFAULT) } - private def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: String): Unit = { + private def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: IndexState): Unit = { openSearchClient.update( new UpdateRequest(testMetadataLogIndex, testLatestId) .doc(latest.copy(state = newState).toJson, XContentType.JSON), diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala new file mode 100644 index 000000000..3c2966144 --- /dev/null +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -0,0 +1,10 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.spark + +class FlintSparkTransactionITSuite { + +} From e6e590483f8bee9195d0c8c7ed4b0e9fb832db5d Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 12:11:47 -0700 Subject: [PATCH 11/22] Add IT base suite Signed-off-by: Chen Dai --- .../metadata/log/FlintMetadataLogEntry.scala | 6 +- .../metadata/log/OptimisticTransaction.java | 6 +- .../OpenSearchOptimisticTransaction.java | 4 +- .../opensearch/flint/spark/FlintSpark.scala | 6 +- .../flint/OpenSearchTransactionSuite.scala | 64 ++++++++++++++ .../FlintOpenSearchTransactionITSuite.scala | 85 +++++++------------ .../spark/FlintSparkTransactionITSuite.scala | 32 ++++++- 7 files changed, 135 insertions(+), 68 deletions(-) create mode 100644 integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index 8b632e4f8..bd6cc98d2 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -27,8 +27,8 @@ import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.I */ case class FlintMetadataLogEntry( id: String, - seqNo: Long, - primaryTerm: Long, + seqNo: Long = -1, + primaryTerm: Long = -1, state: IndexState, dataSource: String, // TODO: get from Spark conf error: String) { @@ -48,7 +48,7 @@ case class FlintMetadataLogEntry( |{ | "version": "1.0", | "type": "flintindexstate", - | "state": "$state", + | "state": "${state.toString}", | "applicationId": "${sys.env.getOrElse("SERVERLESS_EMR_VIRTUAL_CLUSTER_ID", "unknown")}", | "jobId": "${sys.env.getOrElse("SERVERLESS_EMR_JOB_ID", "unknown")}", | "dataSourceName": "$dataSource", diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java index e7bd84b22..2e928a411 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java @@ -45,7 +45,7 @@ public interface OptimisticTransaction { * @param operation operation * @return result */ - T execute(Supplier operation); + T execute(Function operation); /** * No optimistic transaction. @@ -67,8 +67,8 @@ public OptimisticTransaction finalLog(Function operation) { - return operation.get(); + public T execute(Function operation) { + return operation.apply(null); } }; } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index aaef3696a..55553cef9 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -83,7 +83,7 @@ public OpenSearchOptimisticTransaction finalLog(Function operation) { + public T execute(Function operation) { Objects.requireNonNull(initialCondition); Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); @@ -97,7 +97,7 @@ public T execute(Supplier operation) { // TODO: log entry can be same? latest = updateLogEntry(transientAction.apply(latest)); - T result = operation.get(); + T result = operation.apply(latest); updateLogEntry(finalAction.apply(latest)); return result; diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 4b11e8450..ec439b338 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -97,7 +97,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == EMPTY) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(() => flintClient.createIndex(indexName, metadata)) + .execute(latest => flintClient.createIndex(indexName, metadata)) } catch { case e: Exception => logError("Failed to create Flint index", e) } @@ -131,7 +131,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { latest } }) - .execute(() => doRefreshIndex(index, indexName, mode)) + .execute(_ => doRefreshIndex(index, indexName, mode)) } // TODO: move to separate class @@ -247,7 +247,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) .transientLog(latest => latest.copy(state = DELETING)) .finalLog(latest => latest.copy(state = DELETED)) - .execute(() => { + .execute(_ => { // TODO: share same transaction for now stopRefreshingJob(indexName) flintClient.deleteIndex(indexName) diff --git a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala new file mode 100644 index 000000000..b7e951177 --- /dev/null +++ b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint + +import java.util.{Base64, Collections} + +import scala.collection.JavaConverters.mapAsScalaMapConverter + +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest +import org.opensearch.action.get.GetRequest +import org.opensearch.action.index.IndexRequest +import org.opensearch.action.update.UpdateRequest +import org.opensearch.client.RequestOptions +import org.opensearch.client.indices.CreateIndexRequest +import org.opensearch.common.xcontent.XContentType +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.IndexState +import org.opensearch.flint.spark.FlintSparkSuite + +trait OpenSearchTransactionSuite { + self: FlintSparkSuite => + + val testFlintIndex = "flint_test_index" + val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) + val testMetadataLogIndex = ".query_request_history_mys3" + + override def beforeEach(): Unit = { + openSearchClient + .indices() + .create(new CreateIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + } + + override def afterEach(): Unit = { + openSearchClient + .indices() + .delete(new DeleteIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + } + + def latestLogEntry: Map[String, AnyRef] = { + val response = openSearchClient + .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) + + Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap + } + + def createLatestLogEntry(latest: FlintMetadataLogEntry): Unit = { + openSearchClient.index( + new IndexRequest() + .index(testMetadataLogIndex) + .id(testLatestId) + .source(latest.toJson, XContentType.JSON), + RequestOptions.DEFAULT) + } + + def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: IndexState): Unit = { + openSearchClient.update( + new UpdateRequest(testMetadataLogIndex, testLatestId) + .doc(latest.copy(state = newState).toJson, XContentType.JSON), + RequestOptions.DEFAULT) + } +} diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index dc9b9bd50..643b54110 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -5,28 +5,20 @@ package org.opensearch.flint.core -import java.util.{Base64, Collections} +import scala.collection.JavaConverters.mapAsJavaMapConverter -import scala.collection.JavaConverters.{mapAsJavaMapConverter, mapAsScalaMapConverter} - -import org.opensearch.action.admin.indices.delete.DeleteIndexRequest -import org.opensearch.action.get.GetRequest -import org.opensearch.action.index.IndexRequest -import org.opensearch.action.update.UpdateRequest -import org.opensearch.client.RequestOptions -import org.opensearch.client.indices.CreateIndexRequest -import org.opensearch.common.xcontent.XContentType +import org.opensearch.flint.OpenSearchTransactionSuite import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState._ import org.opensearch.flint.core.storage.FlintOpenSearchClient import org.opensearch.flint.spark.FlintSparkSuite import org.scalatest.matchers.should.Matchers -class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { +class FlintOpenSearchTransactionITSuite + extends FlintSparkSuite + with OpenSearchTransactionSuite + with Matchers { - val testFlintIndex = "flint_test_index" - val testMetadataLogIndex = ".query_request_history_mys3" - val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) var flintClient: FlintClient = _ override def beforeAll(): Unit = { @@ -34,47 +26,34 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) } - override def beforeEach(): Unit = { - super.beforeEach() - openSearchClient - .indices() - .create(new CreateIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) - } - - override def afterEach(): Unit = { - openSearchClient - .indices() - .delete(new DeleteIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) - super.afterEach() - } - test("should transit from initial to final if initial is empty") { flintClient .startTransaction(testFlintIndex) .initialLog(latest => { - latest.state shouldBe "empty" + latest.state shouldBe EMPTY true }) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(() => latestLogEntry should contain("state" -> "creating")) + .execute(_ => latestLogEntry should contain("state" -> "creating")) latestLogEntry should contain("state" -> "active") } test("should transit from initial to final if initial is not empty but meet precondition") { // Create doc first to simulate this scenario - createLatestLogEntry(FlintMetadataLogEntry(id = testLatestId, state = "active")) + createLatestLogEntry( + FlintMetadataLogEntry(id = testLatestId, state = ACTIVE, dataSource = "mys3", error = "")) flintClient .startTransaction(testFlintIndex) .initialLog(latest => { - latest.state shouldBe "active" + latest.state shouldBe ACTIVE true }) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(() => latestLogEntry should contain("state" -> "refreshing")) + .execute(_ => latestLogEntry should contain("state" -> "refreshing")) latestLogEntry should contain("state" -> "active") } @@ -86,11 +65,11 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { .initialLog(_ => false) .transientLog(latest => latest) .finalLog(latest => latest) - .execute(() => {}) + .execute(_ => {}) } } - test("should exit if initial log entry updated by others when adding transient log entry") { + test("should fail if initial log entry updated by others when updating transient log entry") { the[IllegalStateException] thrownBy { flintClient .startTransaction(testFlintIndex) @@ -102,30 +81,24 @@ class FlintOpenSearchTransactionITSuite extends FlintSparkSuite with Matchers { latest.copy(state = CREATING) }) .finalLog(latest => latest) - .execute(() => {}) + .execute(_ => {}) } } - private def latestLogEntry: Map[String, AnyRef] = { - val response = openSearchClient - .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) - - Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap - } - - private def createLatestLogEntry(latest: FlintMetadataLogEntry): Unit = { - openSearchClient.index( - new IndexRequest() - .index(testMetadataLogIndex) - .id(testLatestId) - .source(latest.toJson, XContentType.JSON), - RequestOptions.DEFAULT) - } + test("should fail if transient log entry updated by others when updating final log entry") { + the[IllegalStateException] thrownBy { + flintClient + .startTransaction(testFlintIndex) + .initialLog(_ => true) + .transientLog(latest => { - private def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: IndexState): Unit = { - openSearchClient.update( - new UpdateRequest(testMetadataLogIndex, testLatestId) - .doc(latest.copy(state = newState).toJson, XContentType.JSON), - RequestOptions.DEFAULT) + latest.copy(state = CREATING) + }) + .finalLog(latest => latest) + .execute(latest => { + // This update will happen first and thus cause version conflict as expected + updateLatestLogEntry(latest, DELETING) + }) + } } } diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala index 3c2966144..0b64cdbd5 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -5,6 +5,36 @@ package org.opensearch.flint.spark -class FlintSparkTransactionITSuite { +import org.opensearch.flint.OpenSearchTransactionSuite +import org.opensearch.flint.core.FlintVersion +import org.opensearch.flint.core.metadata.FlintMetadata +import org.scalatest.matchers.should.Matchers +import org.apache.spark.sql.{DataFrame, SparkSession} + +class FlintSparkTransactionITSuite + extends FlintSparkSuite + with OpenSearchTransactionSuite + with Matchers { + + /** Test Flint index implementation */ + class FlintSparkFakeIndex extends FlintSparkIndex { + override val kind: String = "fake" + + override val options: FlintSparkIndexOptions = FlintSparkIndexOptions.empty + + override def name(): String = "fake_index" + + override def metadata(): FlintMetadata = + new FlintMetadata(FlintVersion.current(), name(), kind, "source", indexSettings = None) + + override def build(spark: SparkSession, df: Option[DataFrame]): DataFrame = { + null + } + } + + test("create and refresh index") { + + flint.createIndex(new FlintSparkFakeIndex) + } } From 379b78f3360f9b1512321b825416208e8a3e4cf5 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 13:35:14 -0700 Subject: [PATCH 12/22] Extract OS logic to metadata log Signed-off-by: Chen Dai --- .../core/metadata/log/FlintMetadataLog.java | 18 +++ .../metadata/log/FlintMetadataLogEntry.scala | 8 +- .../log/FlintOpenSearchMetadataLog.java | 128 +++++++++++++++++ .../metadata/log/OptimisticTransaction.java | 5 +- .../core/storage/FlintOpenSearchClient.java | 4 +- .../OpenSearchOptimisticTransaction.java | 135 ++++-------------- .../opensearch/flint/spark/FlintSpark.scala | 6 +- .../FlintOpenSearchTransactionITSuite.scala | 19 ++- 8 files changed, 200 insertions(+), 123 deletions(-) create mode 100644 flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java create mode 100644 flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java new file mode 100644 index 000000000..786f4f326 --- /dev/null +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java @@ -0,0 +1,18 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.core.metadata.log; + +import java.util.Optional; + +/** + * Flint metadata log. + */ +public interface FlintMetadataLog { + + T add(T logEntry); + + Optional getLatest(); +} diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index bd6cc98d2..826074c96 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -27,15 +27,15 @@ import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.I */ case class FlintMetadataLogEntry( id: String, - seqNo: Long = -1, - primaryTerm: Long = -1, + seqNo: Long, + primaryTerm: Long, state: IndexState, dataSource: String, // TODO: get from Spark conf error: String) { - def this(docId: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { + def this(id: String, seqNo: Long, primaryTerm: Long, map: java.util.Map[String, AnyRef]) { this( - docId, + id, seqNo, primaryTerm, IndexState.from(map.get("state").asInstanceOf[String]), diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java new file mode 100644 index 000000000..aec6b6cb4 --- /dev/null +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java @@ -0,0 +1,128 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.flint.core.metadata.log; + +import java.io.IOException; +import java.util.Base64; +import java.util.Optional; +import java.util.logging.Logger; +import org.opensearch.OpenSearchException; +import org.opensearch.action.get.GetRequest; +import org.opensearch.action.get.GetResponse; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.WriteRequest; +import org.opensearch.action.update.UpdateRequest; +import org.opensearch.action.update.UpdateResponse; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.RestHighLevelClient; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.flint.core.FlintClient; + +/** + * Flint metadata log in OpenSearch store. + */ +public class FlintOpenSearchMetadataLog implements FlintMetadataLog { + + private static final Logger LOG = Logger.getLogger(FlintOpenSearchMetadataLog.class.getName()); + + /** + * Flint client to create Rest OpenSearch client (This will be refactored later) + */ + private final FlintClient flintClient; + + /** + * Reuse query request index as Flint metadata log store + */ + private final String metadataLogIndexName; + + /** + * Doc id for latest log entry (Naming rule is static so no need to query Flint index metadata) + */ + private final String latestId; + + public FlintOpenSearchMetadataLog(FlintClient flintClient, String flintIndexName, String metadataLogIndexName) { + this.flintClient = flintClient; + this.latestId = Base64.getEncoder().encodeToString(flintIndexName.getBytes()); + this.metadataLogIndexName = metadataLogIndexName; + } + + @Override + public FlintMetadataLogEntry add(FlintMetadataLogEntry logEntry) { + // TODO: use single doc for now. this will be always append in future. + FlintMetadataLogEntry latest; + if (logEntry.id().isEmpty()) { + latest = createLogEntry(logEntry); + } else { + latest = updateLogEntry(logEntry); + } + return latest; + } + + @Override + public Optional getLatest() { + try (RestHighLevelClient client = flintClient.createClient()) { + GetResponse response = + client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); + if (response.isExists()) { + return Optional.of( + new FlintMetadataLogEntry( + response.getId(), + response.getSeqNo(), + response.getPrimaryTerm(), + response.getSourceAsMap())); + } else { + return Optional.empty(); + } + } catch (Exception e) { + throw new IllegalStateException("Failed to fetch latest metadata log entry", e); + } + } + + private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { + LOG.info("Creating log entry " + logEntry); + try (RestHighLevelClient client = flintClient.createClient()) { + logEntry = logEntry.copy( + latestId, + logEntry.seqNo(), logEntry.primaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); + + IndexResponse response = client.index( + new IndexRequest() + .index(metadataLogIndexName) + .id(logEntry.id()) + .source(logEntry.toJson(), XContentType.JSON), + RequestOptions.DEFAULT); + + // Update seqNo and primaryTerm in log entry object + return logEntry.copy( + logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), + logEntry.state(), logEntry.dataSource(), logEntry.error()); + } catch (OpenSearchException | IOException e) { + throw new IllegalStateException("Failed to create initial log entry", e); + } + } + + private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { + LOG.info("Updating log entry " + logEntry); + try (RestHighLevelClient client = flintClient.createClient()) { + UpdateResponse response = + client.update( + new UpdateRequest(metadataLogIndexName, logEntry.id()) + .doc(logEntry.toJson(), XContentType.JSON) + .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(logEntry.seqNo()) + .setIfPrimaryTerm(logEntry.primaryTerm()), + RequestOptions.DEFAULT); + + // Update seqNo and primaryTerm in log entry object + return logEntry.copy( + logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), + logEntry.state(), logEntry.dataSource(), logEntry.error()); + } catch (OpenSearchException | IOException e) { + throw new IllegalStateException("Failed to update log entry: " + logEntry, e); + } + } +} diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java index 2e928a411..3a490a87b 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/OptimisticTransaction.java @@ -7,7 +7,6 @@ import java.util.function.Function; import java.util.function.Predicate; -import java.util.function.Supplier; /** * Optimistic transaction interface that represents a state transition on the state machine. @@ -45,7 +44,7 @@ public interface OptimisticTransaction { * @param operation operation * @return result */ - T execute(Function operation); + T commit(Function operation); /** * No optimistic transaction. @@ -67,7 +66,7 @@ public OptimisticTransaction finalLog(Function operation) { + public T commit(Function operation) { return operation.apply(null); } }; diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 3a3f9abc4..4c0a2875a 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -43,6 +43,7 @@ import org.opensearch.flint.core.FlintOptions; import org.opensearch.flint.core.auth.AWSRequestSigningApacheInterceptor; import org.opensearch.flint.core.metadata.FlintMetadata; +import org.opensearch.flint.core.metadata.log.FlintOpenSearchMetadataLog; import org.opensearch.flint.core.metadata.log.OptimisticTransaction; import org.opensearch.flint.core.metadata.log.OptimisticTransaction.NoOptimisticTransaction; import org.opensearch.index.query.AbstractQueryBuilder; @@ -78,7 +79,8 @@ public FlintOpenSearchClient(FlintOptions options) { String metaLogIndexName = ".query_request_history_mys3"; try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { - return new OpenSearchOptimisticTransaction<>(this, indexName, metaLogIndexName); + return new OpenSearchOptimisticTransaction<>( + new FlintOpenSearchMetadataLog(this, indexName, metaLogIndexName)); } else { return new NoOptimisticTransaction<>(); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index 55553cef9..881778987 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -6,26 +6,14 @@ package org.opensearch.flint.core.storage; import static org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState$; +import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; +import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; -import java.io.IOException; -import java.util.Base64; import java.util.Objects; import java.util.function.Function; import java.util.function.Predicate; -import java.util.function.Supplier; import java.util.logging.Logger; -import org.opensearch.OpenSearchException; -import org.opensearch.action.get.GetRequest; -import org.opensearch.action.get.GetResponse; -import org.opensearch.action.index.IndexRequest; -import org.opensearch.action.index.IndexResponse; -import org.opensearch.action.support.WriteRequest; -import org.opensearch.action.update.UpdateRequest; -import org.opensearch.action.update.UpdateResponse; -import org.opensearch.client.RequestOptions; -import org.opensearch.client.RestHighLevelClient; -import org.opensearch.common.xcontent.XContentType; -import org.opensearch.flint.core.FlintClient; +import org.opensearch.flint.core.metadata.log.FlintMetadataLog; import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry; import org.opensearch.flint.core.metadata.log.OptimisticTransaction; @@ -40,135 +28,70 @@ public class OpenSearchOptimisticTransaction implements OptimisticTransaction private static final Logger LOG = Logger.getLogger(OpenSearchOptimisticTransaction.class.getName()); /** - * Flint client to create Rest OpenSearch client (This will be refactored later) + * Flint metadata log */ - private final FlintClient flintClient; - - /** - * Reuse query request index as Flint metadata log store - */ - private final String metadataLogIndexName; - - /** - * Doc id for latest log entry (Naming rule is static so no need to query Flint index metadata) - */ - private final String latestId; + private final FlintMetadataLog metadataLog; private Predicate initialCondition = null; private Function transientAction = null; private Function finalAction = null; - public OpenSearchOptimisticTransaction(FlintClient flintClient, String flintIndexName, String metadataLogIndexName) { - this.flintClient = flintClient; - this.latestId = Base64.getEncoder().encodeToString(flintIndexName.getBytes()); - this.metadataLogIndexName = metadataLogIndexName; + public OpenSearchOptimisticTransaction( + FlintMetadataLog metadataLog) { + this.metadataLog = metadataLog; } @Override - public OpenSearchOptimisticTransaction initialLog(Predicate initialCondition) { + public OpenSearchOptimisticTransaction initialLog( + Predicate initialCondition) { this.initialCondition = initialCondition; return this; } @Override - public OpenSearchOptimisticTransaction transientLog(Function action) { + public OpenSearchOptimisticTransaction transientLog( + Function action) { this.transientAction = action; return this; } @Override - public OpenSearchOptimisticTransaction finalLog(Function action) { + public OpenSearchOptimisticTransaction finalLog( + Function action) { this.finalAction = action; return this; } @Override - public T execute(Function operation) { + public T commit(Function operation) { Objects.requireNonNull(initialCondition); Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); - FlintMetadataLogEntry latest = getLatestLogEntry(); - if (latest.id().isEmpty()) { - latest = createLogEntry(latest); - } + FlintMetadataLogEntry latest = + metadataLog.getLatest().orElseGet(() -> metadataLog.add(emptyLogEntry())); if (initialCondition.test(latest)) { // TODO: log entry can be same? - latest = updateLogEntry(transientAction.apply(latest)); + latest = metadataLog.add(transientAction.apply(latest)); T result = operation.apply(latest); - updateLogEntry(finalAction.apply(latest)); + metadataLog.add(finalAction.apply(latest)); return result; } else { - throw new IllegalStateException("Exit due to initial log precondition not satisfied"); + throw new IllegalStateException( + "Transaction failed due to initial log precondition not satisfied"); } } - // TODO: Move all these to FlintLogEntry <- FlintOpenSearchLogEntry - - private FlintMetadataLogEntry getLatestLogEntry() { - RestHighLevelClient client = flintClient.createClient(); - try { - GetResponse response = - client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); - - if (response.isExists()) { - return new FlintMetadataLogEntry( - response.getId(), - response.getSeqNo(), - response.getPrimaryTerm(), - response.getSourceAsMap()); - } else { - return new FlintMetadataLogEntry("", -1, -1, IndexState$.MODULE$.EMPTY(), "mys3", ""); - } - } catch (Exception e) { // TODO: resource not found exception? - throw new IllegalStateException("Failed to fetch latest metadata log entry", e); - } - } - - private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { - LOG.info("Creating log entry " + logEntry); - try (RestHighLevelClient client = flintClient.createClient()) { - logEntry = logEntry.copy( - latestId, - logEntry.seqNo(), logEntry.primaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); - - IndexResponse response = client.index( - new IndexRequest() - .index(metadataLogIndexName) - .id(logEntry.id()) - .source(logEntry.toJson(), XContentType.JSON), - RequestOptions.DEFAULT); - - // Update seqNo and primaryTerm in log entry object - return logEntry.copy( - logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), - logEntry.state(), logEntry.dataSource(), logEntry.error()); - } catch (OpenSearchException | IOException e) { - throw new IllegalStateException("Failed to create initial log entry", e); - } - } - - private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { - LOG.info("Updating log entry " + logEntry); - try (RestHighLevelClient client = flintClient.createClient()) { - UpdateResponse response = - client.update( - new UpdateRequest(metadataLogIndexName, logEntry.id()) - .doc(logEntry.toJson(), XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) - .setIfSeqNo(logEntry.seqNo()) - .setIfPrimaryTerm(logEntry.primaryTerm()), - RequestOptions.DEFAULT); - - // Update seqNo and primaryTerm in log entry object - return logEntry.copy( - logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), - logEntry.state(), logEntry.dataSource(), logEntry.error()); - } catch (OpenSearchException | IOException e) { - throw new IllegalStateException("Failed to update log entry: " + logEntry, e); - } + private FlintMetadataLogEntry emptyLogEntry() { + return new FlintMetadataLogEntry( + "", + UNASSIGNED_SEQ_NO, + UNASSIGNED_PRIMARY_TERM, + IndexState$.MODULE$.EMPTY(), + "mys3", // TODO: get it from spark conf + ""); } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index ec439b338..39f25415c 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -97,7 +97,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == EMPTY) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(latest => flintClient.createIndex(indexName, metadata)) + .commit(latest => flintClient.createIndex(indexName, metadata)) } catch { case e: Exception => logError("Failed to create Flint index", e) } @@ -131,7 +131,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { latest } }) - .execute(_ => doRefreshIndex(index, indexName, mode)) + .commit(_ => doRefreshIndex(index, indexName, mode)) } // TODO: move to separate class @@ -247,7 +247,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) .transientLog(latest => latest.copy(state = DELETING)) .finalLog(latest => latest.copy(state = DELETED)) - .execute(_ => { + .commit(_ => { // TODO: share same transaction for now stopRefreshingJob(indexName) flintClient.deleteIndex(indexName) diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index 643b54110..d551b5b89 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -12,6 +12,7 @@ import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState._ import org.opensearch.flint.core.storage.FlintOpenSearchClient import org.opensearch.flint.spark.FlintSparkSuite +import org.opensearch.index.seqno.SequenceNumbers.{UNASSIGNED_PRIMARY_TERM, UNASSIGNED_SEQ_NO} import org.scalatest.matchers.should.Matchers class FlintOpenSearchTransactionITSuite @@ -35,7 +36,7 @@ class FlintOpenSearchTransactionITSuite }) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(_ => latestLogEntry should contain("state" -> "creating")) + .commit(_ => latestLogEntry should contain("state" -> "creating")) latestLogEntry should contain("state" -> "active") } @@ -43,7 +44,13 @@ class FlintOpenSearchTransactionITSuite test("should transit from initial to final if initial is not empty but meet precondition") { // Create doc first to simulate this scenario createLatestLogEntry( - FlintMetadataLogEntry(id = testLatestId, state = ACTIVE, dataSource = "mys3", error = "")) + FlintMetadataLogEntry( + id = testLatestId, + seqNo = UNASSIGNED_SEQ_NO, + primaryTerm = UNASSIGNED_PRIMARY_TERM, + state = ACTIVE, + dataSource = "mys3", + error = "")) flintClient .startTransaction(testFlintIndex) @@ -53,7 +60,7 @@ class FlintOpenSearchTransactionITSuite }) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .execute(_ => latestLogEntry should contain("state" -> "refreshing")) + .commit(_ => latestLogEntry should contain("state" -> "refreshing")) latestLogEntry should contain("state" -> "active") } @@ -65,7 +72,7 @@ class FlintOpenSearchTransactionITSuite .initialLog(_ => false) .transientLog(latest => latest) .finalLog(latest => latest) - .execute(_ => {}) + .commit(_ => {}) } } @@ -81,7 +88,7 @@ class FlintOpenSearchTransactionITSuite latest.copy(state = CREATING) }) .finalLog(latest => latest) - .execute(_ => {}) + .commit(_ => {}) } } @@ -95,7 +102,7 @@ class FlintOpenSearchTransactionITSuite latest.copy(state = CREATING) }) .finalLog(latest => latest) - .execute(latest => { + .commit(latest => { // This update will happen first and thus cause version conflict as expected updateLatestLogEntry(latest, DELETING) }) From 2f26640b95471e1f4d826a37eecd751db3eca6b7 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 13:44:19 -0700 Subject: [PATCH 13/22] Update javadoc and more logging Signed-off-by: Chen Dai --- .../core/metadata/log/FlintMetadataLog.java | 13 +++++++- .../log/FlintOpenSearchMetadataLog.java | 32 +++++++++++++------ .../OpenSearchOptimisticTransaction.java | 4 +-- 3 files changed, 36 insertions(+), 13 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java index 786f4f326..278d078df 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLog.java @@ -8,11 +8,22 @@ import java.util.Optional; /** - * Flint metadata log. + * Flint metadata log that provides transactional support on write API based on different storage. */ public interface FlintMetadataLog { + /** + * Add a new log entry to the metadata log. + * + * @param logEntry log entry + * @return log entry after add + */ T add(T logEntry); + /** + * Get the latest log entry in the metadata log. + * + * @return latest log entry + */ Optional getLatest(); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java index aec6b6cb4..c6383f41a 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java @@ -23,7 +23,8 @@ import org.opensearch.flint.core.FlintClient; /** - * Flint metadata log in OpenSearch store. + * Flint metadata log in OpenSearch store. For now use single doc instead of maintaining history + * of metadata log. */ public class FlintOpenSearchMetadataLog implements FlintMetadataLog { @@ -46,8 +47,8 @@ public class FlintOpenSearchMetadataLog implements FlintMetadataLog getLatest() { + LOG.info("Fetching latest log entry with id " + latestId); try (RestHighLevelClient client = flintClient.createClient()) { GetResponse response = client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); + if (response.isExists()) { - return Optional.of( - new FlintMetadataLogEntry( - response.getId(), - response.getSeqNo(), - response.getPrimaryTerm(), - response.getSourceAsMap())); + FlintMetadataLogEntry latest = new FlintMetadataLogEntry( + response.getId(), + response.getSeqNo(), + response.getPrimaryTerm(), + response.getSourceAsMap()); + + LOG.info("Found latest log entry " + latest); + return Optional.of(latest); } else { + LOG.info("Latest log entry not found"); return Optional.empty(); } } catch (Exception e) { @@ -97,9 +103,12 @@ private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { RequestOptions.DEFAULT); // Update seqNo and primaryTerm in log entry object - return logEntry.copy( + logEntry = logEntry.copy( logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); + + LOG.info("Create log entry " + logEntry); + return logEntry; } catch (OpenSearchException | IOException e) { throw new IllegalStateException("Failed to create initial log entry", e); } @@ -118,9 +127,12 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { RequestOptions.DEFAULT); // Update seqNo and primaryTerm in log entry object - return logEntry.copy( + logEntry = logEntry.copy( logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); + + LOG.info("Log entry updated " + logEntry); + return logEntry; } catch (OpenSearchException | IOException e) { throw new IllegalStateException("Failed to update log entry: " + logEntry, e); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java index 881778987..5ae70fe19 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java @@ -18,8 +18,8 @@ import org.opensearch.flint.core.metadata.log.OptimisticTransaction; /** - * Optimistic transaction implementation by OpenSearch OCC. - * For now use single doc instead of maintaining history of metadata log. + * Default optimistic transaction implementation that captures the basic workflow for + * transaction support by optimistic locking. * * @param result type */ From d19c0fab82c8dfb85cda3733411e0a8d500d4c12 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 14:12:11 -0700 Subject: [PATCH 14/22] Add more IT with skipping index Signed-off-by: Chen Dai --- .../flint/OpenSearchTransactionSuite.scala | 12 ++-- .../FlintOpenSearchTransactionITSuite.scala | 17 +++-- .../spark/FlintSparkTransactionITSuite.scala | 67 ++++++++++++++----- 3 files changed, 67 insertions(+), 29 deletions(-) diff --git a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala index b7e951177..5fecc31ae 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala @@ -5,7 +5,7 @@ package org.opensearch.flint -import java.util.{Base64, Collections} +import java.util.Collections import scala.collection.JavaConverters.mapAsScalaMapConverter @@ -23,8 +23,6 @@ import org.opensearch.flint.spark.FlintSparkSuite trait OpenSearchTransactionSuite { self: FlintSparkSuite => - val testFlintIndex = "flint_test_index" - val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) val testMetadataLogIndex = ".query_request_history_mys3" override def beforeEach(): Unit = { @@ -39,9 +37,9 @@ trait OpenSearchTransactionSuite { .delete(new DeleteIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) } - def latestLogEntry: Map[String, AnyRef] = { + def latestLogEntry(latestId: String): Map[String, AnyRef] = { val response = openSearchClient - .get(new GetRequest(testMetadataLogIndex, testLatestId), RequestOptions.DEFAULT) + .get(new GetRequest(testMetadataLogIndex, latestId), RequestOptions.DEFAULT) Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap } @@ -50,14 +48,14 @@ trait OpenSearchTransactionSuite { openSearchClient.index( new IndexRequest() .index(testMetadataLogIndex) - .id(testLatestId) + .id(latest.id) .source(latest.toJson, XContentType.JSON), RequestOptions.DEFAULT) } def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: IndexState): Unit = { openSearchClient.update( - new UpdateRequest(testMetadataLogIndex, testLatestId) + new UpdateRequest(testMetadataLogIndex, latest.id) .doc(latest.copy(state = newState).toJson, XContentType.JSON), RequestOptions.DEFAULT) } diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala index d551b5b89..f0e002a55 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala @@ -5,6 +5,8 @@ package org.opensearch.flint.core +import java.util.Base64 + import scala.collection.JavaConverters.mapAsJavaMapConverter import org.opensearch.flint.OpenSearchTransactionSuite @@ -20,6 +22,8 @@ class FlintOpenSearchTransactionITSuite with OpenSearchTransactionSuite with Matchers { + val testFlintIndex = "flint_test_index" + val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) var flintClient: FlintClient = _ override def beforeAll(): Unit = { @@ -27,7 +31,7 @@ class FlintOpenSearchTransactionITSuite flintClient = new FlintOpenSearchClient(new FlintOptions(openSearchOptions.asJava)) } - test("should transit from initial to final if initial is empty") { + test("should transit from initial to final log if initial log is empty") { flintClient .startTransaction(testFlintIndex) .initialLog(latest => { @@ -36,12 +40,13 @@ class FlintOpenSearchTransactionITSuite }) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .commit(_ => latestLogEntry should contain("state" -> "creating")) + .commit(_ => latestLogEntry(testLatestId) should contain("state" -> "creating")) - latestLogEntry should contain("state" -> "active") + latestLogEntry(testLatestId) should contain("state" -> "active") } - test("should transit from initial to final if initial is not empty but meet precondition") { + test( + "should transit from initial to final log if initial is not empty but precondition satisfied") { // Create doc first to simulate this scenario createLatestLogEntry( FlintMetadataLogEntry( @@ -60,9 +65,9 @@ class FlintOpenSearchTransactionITSuite }) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .commit(_ => latestLogEntry should contain("state" -> "refreshing")) + .commit(_ => latestLogEntry(testLatestId) should contain("state" -> "refreshing")) - latestLogEntry should contain("state" -> "active") + latestLogEntry(testLatestId) should contain("state" -> "active") } test("should exit if initial log entry doesn't meet precondition") { diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala index 0b64cdbd5..140046e96 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -5,36 +5,71 @@ package org.opensearch.flint.spark +import java.util.Base64 + import org.opensearch.flint.OpenSearchTransactionSuite -import org.opensearch.flint.core.FlintVersion -import org.opensearch.flint.core.metadata.FlintMetadata +import org.opensearch.flint.spark.FlintSpark.RefreshMode.{FULL, INCREMENTAL} +import org.opensearch.flint.spark.skipping.FlintSparkSkippingIndex.getSkippingIndexName import org.scalatest.matchers.should.Matchers -import org.apache.spark.sql.{DataFrame, SparkSession} - class FlintSparkTransactionITSuite extends FlintSparkSuite with OpenSearchTransactionSuite with Matchers { - /** Test Flint index implementation */ - class FlintSparkFakeIndex extends FlintSparkIndex { - override val kind: String = "fake" + /** Test table and index name */ + private val testTable = "spark_catalog.default.test" + private val testFlintIndex = getSkippingIndexName(testTable) + private val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) + + override def beforeAll(): Unit = { + super.beforeAll() + createPartitionedTable(testTable) + } + + override def afterEach(): Unit = { + super.afterEach() + flint.deleteIndex(testFlintIndex) + } - override val options: FlintSparkIndexOptions = FlintSparkIndexOptions.empty + test("create index") { + flint + .skippingIndex() + .onTable(testTable) + .addPartitions("year", "month") + .create() + latestLogEntry(testLatestId) should contain("state" -> "active") + } - override def name(): String = "fake_index" + test("manual refresh index") { + flint + .skippingIndex() + .onTable(testTable) + .addPartitions("year", "month") + .create() + flint.refreshIndex(testFlintIndex, FULL) - override def metadata(): FlintMetadata = - new FlintMetadata(FlintVersion.current(), name(), kind, "source", indexSettings = None) + latestLogEntry(testLatestId) should contain("state" -> "active") + } - override def build(spark: SparkSession, df: Option[DataFrame]): DataFrame = { - null - } + test("incremental refresh index") { + flint + .skippingIndex() + .onTable(testTable) + .addPartitions("year", "month") + .create() + flint.refreshIndex(testFlintIndex, INCREMENTAL) + latestLogEntry(testLatestId) should contain("state" -> "refreshing") } - test("create and refresh index") { + test("delete index") { + flint + .skippingIndex() + .onTable(testTable) + .addPartitions("year", "month") + .create() + flint.deleteIndex(testFlintIndex) - flint.createIndex(new FlintSparkFakeIndex) + latestLogEntry(testLatestId) should contain("state" -> "deleted") } } From a5af650728a234035fdd56d13ebb39f538d00ed5 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 15:15:50 -0700 Subject: [PATCH 15/22] Refactor and add logging for manual test Signed-off-by: Chen Dai --- .../log/DefaultOptimisticTransaction.java} | 30 +++++++++++-------- .../core/storage/FlintOpenSearchClient.java | 4 +-- .../FlintOpenSearchMetadataLog.java | 16 +++++----- .../opensearch/flint/spark/FlintSpark.scala | 26 ++++++++++++++-- .../flint/OpenSearchTransactionSuite.scala | 4 +++ ...te.scala => FlintTransactionITSuite.scala} | 12 +++++++- 6 files changed, 67 insertions(+), 25 deletions(-) rename flint-core/src/main/scala/org/opensearch/flint/core/{storage/OpenSearchOptimisticTransaction.java => metadata/log/DefaultOptimisticTransaction.java} (74%) rename flint-core/src/main/scala/org/opensearch/flint/core/{metadata/log => storage}/FlintOpenSearchMetadataLog.java (91%) rename integ-test/src/test/scala/org/opensearch/flint/core/{FlintOpenSearchTransactionITSuite.scala => FlintTransactionITSuite.scala} (90%) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java similarity index 74% rename from flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java rename to flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java index 5ae70fe19..ffb948b2f 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/OpenSearchOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.flint.core.storage; +package org.opensearch.flint.core.metadata.log; import static org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState$; import static org.opensearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; @@ -13,9 +13,6 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.logging.Logger; -import org.opensearch.flint.core.metadata.log.FlintMetadataLog; -import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry; -import org.opensearch.flint.core.metadata.log.OptimisticTransaction; /** * Default optimistic transaction implementation that captures the basic workflow for @@ -23,9 +20,9 @@ * * @param result type */ -public class OpenSearchOptimisticTransaction implements OptimisticTransaction { +public class DefaultOptimisticTransaction implements OptimisticTransaction { - private static final Logger LOG = Logger.getLogger(OpenSearchOptimisticTransaction.class.getName()); + private static final Logger LOG = Logger.getLogger(DefaultOptimisticTransaction.class.getName()); /** * Flint metadata log @@ -36,27 +33,27 @@ public class OpenSearchOptimisticTransaction implements OptimisticTransaction private Function transientAction = null; private Function finalAction = null; - public OpenSearchOptimisticTransaction( + public DefaultOptimisticTransaction( FlintMetadataLog metadataLog) { this.metadataLog = metadataLog; } @Override - public OpenSearchOptimisticTransaction initialLog( + public DefaultOptimisticTransaction initialLog( Predicate initialCondition) { this.initialCondition = initialCondition; return this; } @Override - public OpenSearchOptimisticTransaction transientLog( + public DefaultOptimisticTransaction transientLog( Function action) { this.transientAction = action; return this; } @Override - public OpenSearchOptimisticTransaction finalLog( + public DefaultOptimisticTransaction finalLog( Function action) { this.finalAction = action; return this; @@ -65,21 +62,28 @@ public OpenSearchOptimisticTransaction finalLog( @Override public T commit(Function operation) { Objects.requireNonNull(initialCondition); - Objects.requireNonNull(transientAction); Objects.requireNonNull(finalAction); + // Get the latest log and create if not exists FlintMetadataLogEntry latest = metadataLog.getLatest().orElseGet(() -> metadataLog.add(emptyLogEntry())); + // Perform initial log check if (initialCondition.test(latest)) { - // TODO: log entry can be same? - latest = metadataLog.add(transientAction.apply(latest)); + // Append optional transient log + if (transientAction != null) { + latest = metadataLog.add(transientAction.apply(latest)); + } + + // Perform operation T result = operation.apply(latest); + // Append final log metadataLog.add(finalAction.apply(latest)); return result; } else { + LOG.warning("Initial log entry doesn't satisfy precondition " + latest); throw new IllegalStateException( "Transaction failed due to initial log precondition not satisfied"); } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 4c0a2875a..3478032cb 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -43,7 +43,7 @@ import org.opensearch.flint.core.FlintOptions; import org.opensearch.flint.core.auth.AWSRequestSigningApacheInterceptor; import org.opensearch.flint.core.metadata.FlintMetadata; -import org.opensearch.flint.core.metadata.log.FlintOpenSearchMetadataLog; +import org.opensearch.flint.core.metadata.log.DefaultOptimisticTransaction; import org.opensearch.flint.core.metadata.log.OptimisticTransaction; import org.opensearch.flint.core.metadata.log.OptimisticTransaction.NoOptimisticTransaction; import org.opensearch.index.query.AbstractQueryBuilder; @@ -79,7 +79,7 @@ public FlintOpenSearchClient(FlintOptions options) { String metaLogIndexName = ".query_request_history_mys3"; try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { - return new OpenSearchOptimisticTransaction<>( + return new DefaultOptimisticTransaction<>( new FlintOpenSearchMetadataLog(this, indexName, metaLogIndexName)); } else { return new NoOptimisticTransaction<>(); diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java similarity index 91% rename from flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java rename to flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java index c6383f41a..7cf916a4a 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintOpenSearchMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.flint.core.metadata.log; +package org.opensearch.flint.core.storage; import java.io.IOException; import java.util.Base64; @@ -21,6 +21,8 @@ import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.xcontent.XContentType; import org.opensearch.flint.core.FlintClient; +import org.opensearch.flint.core.metadata.log.FlintMetadataLog; +import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry; /** * Flint metadata log in OpenSearch store. For now use single doc instead of maintaining history @@ -38,16 +40,16 @@ public class FlintOpenSearchMetadataLog implements FlintMetadataLog getLatest() { LOG.info("Fetching latest log entry with id " + latestId); try (RestHighLevelClient client = flintClient.createClient()) { GetResponse response = - client.get(new GetRequest(metadataLogIndexName, latestId), RequestOptions.DEFAULT); + client.get(new GetRequest(indexName, latestId), RequestOptions.DEFAULT); if (response.isExists()) { FlintMetadataLogEntry latest = new FlintMetadataLogEntry( @@ -97,7 +99,7 @@ private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { IndexResponse response = client.index( new IndexRequest() - .index(metadataLogIndexName) + .index(indexName) .id(logEntry.id()) .source(logEntry.toJson(), XContentType.JSON), RequestOptions.DEFAULT); @@ -119,7 +121,7 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { try (RestHighLevelClient client = flintClient.createClient()) { UpdateResponse response = client.update( - new UpdateRequest(metadataLogIndexName, logEntry.id()) + new UpdateRequest(indexName, logEntry.id()) .doc(logEntry.toJson(), XContentType.JSON) .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) .setIfSeqNo(logEntry.seqNo()) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 39f25415c..10309a8e2 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -5,6 +5,8 @@ package org.opensearch.flint.spark +import java.util.concurrent.{Executors, ScheduledExecutorService, TimeUnit} + import scala.collection.JavaConverters._ import org.json4s.{Formats, NoTypeHints} @@ -44,6 +46,9 @@ class FlintSpark(val spark: SparkSession) extends Logging { /** Required by json4s parse function */ implicit val formats: Formats = Serialization.formats(NoTypeHints) + SkippingKindSerializer + /** Scheduler for updating index state regularly as needed, such as incremental refreshing */ + private val executor: ScheduledExecutorService = Executors.newScheduledThreadPool(1) + /** * Create index builder for creating index with fluent API. * @@ -99,7 +104,10 @@ class FlintSpark(val spark: SparkSession) extends Logging { .finalLog(latest => latest.copy(state = ACTIVE)) .commit(latest => flintClient.createIndex(indexName, metadata)) } catch { - case e: Exception => logError("Failed to create Flint index", e) + case e: Exception => + logError("Failed to create Flint index", e) + throw new IllegalStateException( + "Failed to create Flint index due to transaction failure") } } } @@ -124,10 +132,24 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == ACTIVE) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => { + // Change state to active if full, otherwise update index state regularly if (mode == FULL) { latest.copy(state = ACTIVE) } else { - // TODO: scheduling regular update on heartbeat timestamp + executor.scheduleAtFixedRate( + () => { + logInfo("Scheduler triggers log entry update") + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == REFRESHING) + .finalLog(latest => latest) // timestamp will update automatically + .commit(latest => logInfo("Updating log entry to " + latest)) + }, + 15, // Delay to ensure final logging is complete first, otherwise version conflicts + 60, // TODO: make interval configurable + TimeUnit.SECONDS) + + // Return log entry and keep its state as refreshing latest } }) diff --git a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala index 5fecc31ae..337e5357b 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala @@ -20,6 +20,10 @@ import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.IndexState import org.opensearch.flint.spark.FlintSparkSuite +/** + * Transaction test base suite that creates the metadata log index which enables transaction + * support in index operation. + */ trait OpenSearchTransactionSuite { self: FlintSparkSuite => diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala similarity index 90% rename from integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala rename to integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala index f0e002a55..49bcb7706 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala @@ -17,7 +17,7 @@ import org.opensearch.flint.spark.FlintSparkSuite import org.opensearch.index.seqno.SequenceNumbers.{UNASSIGNED_PRIMARY_TERM, UNASSIGNED_SEQ_NO} import org.scalatest.matchers.should.Matchers -class FlintOpenSearchTransactionITSuite +class FlintTransactionITSuite extends FlintSparkSuite with OpenSearchTransactionSuite with Matchers { @@ -45,6 +45,16 @@ class FlintOpenSearchTransactionITSuite latestLogEntry(testLatestId) should contain("state" -> "active") } + test("should transit from initial to final log directly if no transient log") { + flintClient + .startTransaction(testFlintIndex) + .initialLog(_ => true) + .finalLog(latest => latest.copy(state = ACTIVE)) + .commit(_ => latestLogEntry(testLatestId) should contain("state" -> "empty")) + + latestLogEntry(testLatestId) should contain("state" -> "active") + } + test( "should transit from initial to final log if initial is not empty but precondition satisfied") { // Create doc first to simulate this scenario From 91be3799966b2400f203a704547357fc5ac69647 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 15:28:46 -0700 Subject: [PATCH 16/22] Refactor and add logging for manual test Signed-off-by: Chen Dai --- .../opensearch/flint/spark/FlintSpark.scala | 212 ++++++++++-------- 1 file changed, 116 insertions(+), 96 deletions(-) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 10309a8e2..03f0ec5d3 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -106,8 +106,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { } catch { case e: Exception => logError("Failed to create Flint index", e) - throw new IllegalStateException( - "Failed to create Flint index due to transaction failure") + throw new IllegalStateException("Failed to create Flint index") } } } @@ -127,90 +126,26 @@ class FlintSpark(val spark: SparkSession) extends Logging { val index = describeIndex(indexName) .getOrElse(throw new IllegalStateException(s"Index $indexName doesn't exist")) - flintClient - .startTransaction(indexName) - .initialLog(latest => latest.state == ACTIVE) - .transientLog(latest => latest.copy(state = REFRESHING)) - .finalLog(latest => { - // Change state to active if full, otherwise update index state regularly - if (mode == FULL) { - latest.copy(state = ACTIVE) - } else { - executor.scheduleAtFixedRate( - () => { - logInfo("Scheduler triggers log entry update") - flintClient - .startTransaction(indexName) - .initialLog(latest => latest.state == REFRESHING) - .finalLog(latest => latest) // timestamp will update automatically - .commit(latest => logInfo("Updating log entry to " + latest)) - }, - 15, // Delay to ensure final logging is complete first, otherwise version conflicts - 60, // TODO: make interval configurable - TimeUnit.SECONDS) - - // Return log entry and keep its state as refreshing - latest - } - }) - .commit(_ => doRefreshIndex(index, indexName, mode)) - } - - // TODO: move to separate class - def doRefreshIndex( - index: FlintSparkIndex, - indexName: String, - mode: RefreshMode): Option[String] = { - val options = index.options - val tableName = index.metadata().source - - // Batch refresh Flint index from the given source data frame - def batchRefresh(df: Option[DataFrame] = None): Unit = { - index - .build(spark, df) - .write - .format(FLINT_DATASOURCE) - .options(flintSparkConf.properties) - .mode(Overwrite) - .save(indexName) - } - - mode match { - case FULL if isIncrementalRefreshing(indexName) => - throw new IllegalStateException( - s"Index $indexName is incremental refreshing and cannot be manual refreshed") - - case FULL => - batchRefresh() - None - - // Flint index has specialized logic and capability for incremental refresh - case INCREMENTAL if index.isInstanceOf[StreamingRefresh] => - val job = - index - .asInstanceOf[StreamingRefresh] - .buildStream(spark) - .writeStream - .queryName(indexName) - .format(FLINT_DATASOURCE) - .options(flintSparkConf.properties) - .addSinkOptions(options) - .start(indexName) - Some(job.id.toString) - - // Otherwise, fall back to foreachBatch + batch refresh - case INCREMENTAL => - val job = spark.readStream - .options(options.extraSourceOptions(tableName)) - .table(tableName) - .writeStream - .queryName(indexName) - .addSinkOptions(options) - .foreachBatch { (batchDF: DataFrame, _: Long) => - batchRefresh(Some(batchDF)) + try { + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == ACTIVE) + .transientLog(latest => latest.copy(state = REFRESHING)) + .finalLog(latest => { + // Change state to active if full, otherwise update index state regularly + if (mode == FULL) { + latest.copy(state = ACTIVE) + } else { + // Schedule regular update and return log entry as refreshing state + scheduleIndexStateUpdate(indexName) + latest } - .start() - Some(job.id.toString) + }) + .commit(_ => doRefreshIndex(index, indexName, mode)) + } catch { + case e: Exception => + logError("Failed to refresh Flint index", e) + throw new IllegalStateException("Failed to refresh Flint index") } } @@ -264,17 +199,23 @@ class FlintSpark(val spark: SparkSession) extends Logging { def deleteIndex(indexName: String): Boolean = { logInfo(s"Deleting Flint index $indexName") if (flintClient.exists(indexName)) { - flintClient - .startTransaction(indexName) - .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) - .transientLog(latest => latest.copy(state = DELETING)) - .finalLog(latest => latest.copy(state = DELETED)) - .commit(_ => { - // TODO: share same transaction for now - stopRefreshingJob(indexName) - flintClient.deleteIndex(indexName) - }) - true + try { + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) + .transientLog(latest => latest.copy(state = DELETING)) + .finalLog(latest => latest.copy(state = DELETED)) + .commit(_ => { + // TODO: share same transaction for now + stopRefreshingJob(indexName) + flintClient.deleteIndex(indexName) + true + }) + } catch { + case e: Exception => + logError("Failed to delete Flint index", e) + throw new IllegalStateException("Failed to delete Flint index") + } } else { false } @@ -295,6 +236,85 @@ class FlintSpark(val spark: SparkSession) extends Logging { private def isIncrementalRefreshing(indexName: String): Boolean = spark.streams.active.exists(_.name == indexName) + private def scheduleIndexStateUpdate(indexName: String): Unit = { + executor.scheduleAtFixedRate( + () => { + logInfo("Scheduler triggers index log entry update") + try { + flintClient + .startTransaction(indexName) + .initialLog(latest => latest.state == REFRESHING) + .finalLog(latest => latest) // timestamp will update automatically + .commit(latest => logInfo("Updating log entry to " + latest)) + } catch { + case e: Exception => + logError("Failed to update index log entry", e) + throw new IllegalStateException("Failed to update index log entry") + } + }, + 15, // Delay to ensure final logging is complete first, otherwise version conflicts + 60, // TODO: make interval configurable + TimeUnit.SECONDS) + } + + // TODO: move to separate class + private def doRefreshIndex( + index: FlintSparkIndex, + indexName: String, + mode: RefreshMode): Option[String] = { + val options = index.options + val tableName = index.metadata().source + + // Batch refresh Flint index from the given source data frame + def batchRefresh(df: Option[DataFrame] = None): Unit = { + index + .build(spark, df) + .write + .format(FLINT_DATASOURCE) + .options(flintSparkConf.properties) + .mode(Overwrite) + .save(indexName) + } + + mode match { + case FULL if isIncrementalRefreshing(indexName) => + throw new IllegalStateException( + s"Index $indexName is incremental refreshing and cannot be manual refreshed") + + case FULL => + batchRefresh() + None + + // Flint index has specialized logic and capability for incremental refresh + case INCREMENTAL if index.isInstanceOf[StreamingRefresh] => + val job = + index + .asInstanceOf[StreamingRefresh] + .buildStream(spark) + .writeStream + .queryName(indexName) + .format(FLINT_DATASOURCE) + .options(flintSparkConf.properties) + .addSinkOptions(options) + .start(indexName) + Some(job.id.toString) + + // Otherwise, fall back to foreachBatch + batch refresh + case INCREMENTAL => + val job = spark.readStream + .options(options.extraSourceOptions(tableName)) + .table(tableName) + .writeStream + .queryName(indexName) + .addSinkOptions(options) + .foreachBatch { (batchDF: DataFrame, _: Long) => + batchRefresh(Some(batchDF)) + } + .start() + Some(job.id.toString) + } + } + private def stopRefreshingJob(indexName: String): Unit = { logInfo(s"Terminating refreshing job $indexName") val job = spark.streams.active.find(_.name == indexName) From 943480aa7365445d9bae4b263edbe63986789aa9 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 16:12:40 -0700 Subject: [PATCH 17/22] Read metadata log index name from Spark conf Signed-off-by: Chen Dai --- .../opensearch/flint/core/FlintClient.java | 3 +- .../metadata/log/FlintMetadataLogEntry.scala | 3 +- .../core/storage/FlintOpenSearchClient.java | 5 +- .../storage/FlintOpenSearchMetadataLog.java | 55 ++++++++++++++++--- .../opensearch/flint/spark/FlintSpark.scala | 17 ++++-- .../flint/OpenSearchTransactionSuite.scala | 22 +++++--- .../core/FlintOpenSearchClientSuite.scala | 2 +- .../flint/core/FlintTransactionITSuite.scala | 18 +++--- .../spark/FlintSparkTransactionITSuite.scala | 3 +- 9 files changed, 88 insertions(+), 40 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java index 9706525b7..17358b9a9 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java @@ -23,9 +23,10 @@ public interface FlintClient { * Start a new optimistic transaction. * * @param indexName index name + * @param metaLogIndexName metadata log index name * @return transaction handle */ - OptimisticTransaction startTransaction(String indexName); + OptimisticTransaction startTransaction(String indexName, String metaLogIndexName); /** * Create a Flint index with the metadata given. diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala index 826074c96..8b67ac2ae 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/FlintMetadataLogEntry.scala @@ -44,11 +44,12 @@ case class FlintMetadataLogEntry( } def toJson: String = { + // Implicitly populate latest appId, jobId and timestamp whenever persist s""" |{ | "version": "1.0", | "type": "flintindexstate", - | "state": "${state.toString}", + | "state": "$state", | "applicationId": "${sys.env.getOrElse("SERVERLESS_EMR_VIRTUAL_CLUSTER_ID", "unknown")}", | "jobId": "${sys.env.getOrElse("SERVERLESS_EMR_JOB_ID", "unknown")}", | "dataSourceName": "$dataSource", diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 3478032cb..a456757b7 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -74,9 +74,8 @@ public FlintOpenSearchClient(FlintOptions options) { this.options = options; } - @Override public OptimisticTransaction startTransaction(String indexName) { - LOG.info("Starting transaction on index " + indexName); - String metaLogIndexName = ".query_request_history_mys3"; + @Override public OptimisticTransaction startTransaction(String indexName, String metaLogIndexName) { + LOG.info("Starting transaction on index " + indexName + " and metadata log index " + metaLogIndexName); try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { return new DefaultOptimisticTransaction<>( diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java index 7cf916a4a..ba9824c4f 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java @@ -8,8 +8,10 @@ import java.io.IOException; import java.util.Base64; import java.util.Optional; +import java.util.function.Function; import java.util.logging.Logger; import org.opensearch.OpenSearchException; +import org.opensearch.action.DocWriteResponse; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; @@ -93,9 +95,14 @@ public Optional getLatest() { private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { LOG.info("Creating log entry " + logEntry); try (RestHighLevelClient client = flintClient.createClient()) { + // Assign doc ID here logEntry = logEntry.copy( latestId, - logEntry.seqNo(), logEntry.primaryTerm(), logEntry.state(), logEntry.dataSource(), logEntry.error()); + logEntry.seqNo(), + logEntry.primaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error()); IndexResponse response = client.index( new IndexRequest() @@ -104,12 +111,16 @@ private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { .source(logEntry.toJson(), XContentType.JSON), RequestOptions.DEFAULT); - // Update seqNo and primaryTerm in log entry object + // Update seqNo and primaryTerm in log entry logEntry = logEntry.copy( - logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), - logEntry.state(), logEntry.dataSource(), logEntry.error()); - - LOG.info("Create log entry " + logEntry); + logEntry.id(), + response.getSeqNo(), + response.getPrimaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error()); + + LOG.info("Log entry created " + logEntry); return logEntry; } catch (OpenSearchException | IOException e) { throw new IllegalStateException("Failed to create initial log entry", e); @@ -128,10 +139,14 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { .setIfPrimaryTerm(logEntry.primaryTerm()), RequestOptions.DEFAULT); - // Update seqNo and primaryTerm in log entry object + // Update seqNo and primaryTerm in log entry logEntry = logEntry.copy( - logEntry.id(), response.getSeqNo(), response.getPrimaryTerm(), - logEntry.state(), logEntry.dataSource(), logEntry.error()); + logEntry.id(), + response.getSeqNo(), + response.getPrimaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error()); LOG.info("Log entry updated " + logEntry); return logEntry; @@ -139,4 +154,26 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { throw new IllegalStateException("Failed to update log entry: " + logEntry, e); } } + + private FlintMetadataLogEntry writeLogEntry( + FlintMetadataLogEntry logEntry, + Function write) { + try (RestHighLevelClient client = flintClient.createClient()) { + DocWriteResponse response = write.apply(client); + + // Update seqNo and primaryTerm in log entry + logEntry = logEntry.copy( + logEntry.id(), + response.getSeqNo(), + response.getPrimaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error()); + + LOG.info("Log entry written as " + logEntry); + return logEntry; + } catch (OpenSearchException | IOException e) { + throw new IllegalStateException("Failed to write log entry " + logEntry, e); + } + } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 03f0ec5d3..332b9618e 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -49,6 +49,15 @@ class FlintSpark(val spark: SparkSession) extends Logging { /** Scheduler for updating index state regularly as needed, such as incremental refreshing */ private val executor: ScheduledExecutorService = Executors.newScheduledThreadPool(1) + /** + * Metadata log index name with a default name for backward compatibility. If the index doesn't + * exist, the transaction support will be disabled in FlintClient. + */ + private val metaLogIndexName: String = { + val indexName = spark.conf.getOption("spark.flint.job.requestIndex") + indexName.getOrElse(".query_execution_request") + } + /** * Create index builder for creating index with fluent API. * @@ -98,7 +107,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { val metadata = index.metadata() try { flintClient - .startTransaction(indexName) + .startTransaction(indexName, metaLogIndexName) .initialLog(latest => latest.state == EMPTY) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) @@ -128,7 +137,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { try { flintClient - .startTransaction(indexName) + .startTransaction(indexName, metaLogIndexName) .initialLog(latest => latest.state == ACTIVE) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => { @@ -201,7 +210,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { if (flintClient.exists(indexName)) { try { flintClient - .startTransaction(indexName) + .startTransaction(indexName, metaLogIndexName) .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) .transientLog(latest => latest.copy(state = DELETING)) .finalLog(latest => latest.copy(state = DELETED)) @@ -242,7 +251,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { logInfo("Scheduler triggers index log entry update") try { flintClient - .startTransaction(indexName) + .startTransaction(indexName, metaLogIndexName) .initialLog(latest => latest.state == REFRESHING) .finalLog(latest => latest) // timestamp will update automatically .commit(latest => logInfo("Updating log entry to " + latest)) diff --git a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala index 337e5357b..f5ae10b5e 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala @@ -24,26 +24,32 @@ import org.opensearch.flint.spark.FlintSparkSuite * Transaction test base suite that creates the metadata log index which enables transaction * support in index operation. */ -trait OpenSearchTransactionSuite { - self: FlintSparkSuite => +trait OpenSearchTransactionSuite extends FlintSparkSuite { - val testMetadataLogIndex = ".query_request_history_mys3" + val testMetaLogIndex = ".query_execution_request_mys3" + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set("spark.flint.job.requestIndex", testMetaLogIndex) + } override def beforeEach(): Unit = { + super.beforeEach() openSearchClient .indices() - .create(new CreateIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + .create(new CreateIndexRequest(testMetaLogIndex), RequestOptions.DEFAULT) } override def afterEach(): Unit = { openSearchClient .indices() - .delete(new DeleteIndexRequest(testMetadataLogIndex), RequestOptions.DEFAULT) + .delete(new DeleteIndexRequest(testMetaLogIndex), RequestOptions.DEFAULT) + super.afterEach() } def latestLogEntry(latestId: String): Map[String, AnyRef] = { val response = openSearchClient - .get(new GetRequest(testMetadataLogIndex, latestId), RequestOptions.DEFAULT) + .get(new GetRequest(testMetaLogIndex, latestId), RequestOptions.DEFAULT) Option(response.getSourceAsMap).getOrElse(Collections.emptyMap()).asScala.toMap } @@ -51,7 +57,7 @@ trait OpenSearchTransactionSuite { def createLatestLogEntry(latest: FlintMetadataLogEntry): Unit = { openSearchClient.index( new IndexRequest() - .index(testMetadataLogIndex) + .index(testMetaLogIndex) .id(latest.id) .source(latest.toJson, XContentType.JSON), RequestOptions.DEFAULT) @@ -59,7 +65,7 @@ trait OpenSearchTransactionSuite { def updateLatestLogEntry(latest: FlintMetadataLogEntry, newState: IndexState): Unit = { openSearchClient.update( - new UpdateRequest(testMetadataLogIndex, latest.id) + new UpdateRequest(testMetaLogIndex, latest.id) .doc(latest.copy(state = newState).toJson, XContentType.JSON), RequestOptions.DEFAULT) } diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala index e096723a4..060c57940 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintOpenSearchClientSuite.scala @@ -32,7 +32,7 @@ class FlintOpenSearchClientSuite extends AnyFlatSpec with OpenSearchSuite with M behavior of "Flint OpenSearch client" it should "start no optimistic transaction if metadata log index doesn't exists" in { - val transaction = flintClient.startTransaction("test") + val transaction = flintClient.startTransaction("test", "non-exist-index") transaction shouldBe a[NoOptimisticTransaction[AnyRef]] } diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala index 49bcb7706..29eae82f4 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala @@ -13,14 +13,10 @@ import org.opensearch.flint.OpenSearchTransactionSuite import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState._ import org.opensearch.flint.core.storage.FlintOpenSearchClient -import org.opensearch.flint.spark.FlintSparkSuite import org.opensearch.index.seqno.SequenceNumbers.{UNASSIGNED_PRIMARY_TERM, UNASSIGNED_SEQ_NO} import org.scalatest.matchers.should.Matchers -class FlintTransactionITSuite - extends FlintSparkSuite - with OpenSearchTransactionSuite - with Matchers { +class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { val testFlintIndex = "flint_test_index" val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) @@ -33,7 +29,7 @@ class FlintTransactionITSuite test("should transit from initial to final log if initial log is empty") { flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(latest => { latest.state shouldBe EMPTY true @@ -47,7 +43,7 @@ class FlintTransactionITSuite test("should transit from initial to final log directly if no transient log") { flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(_ => true) .finalLog(latest => latest.copy(state = ACTIVE)) .commit(_ => latestLogEntry(testLatestId) should contain("state" -> "empty")) @@ -68,7 +64,7 @@ class FlintTransactionITSuite error = "")) flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(latest => { latest.state shouldBe ACTIVE true @@ -83,7 +79,7 @@ class FlintTransactionITSuite test("should exit if initial log entry doesn't meet precondition") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(_ => false) .transientLog(latest => latest) .finalLog(latest => latest) @@ -94,7 +90,7 @@ class FlintTransactionITSuite test("should fail if initial log entry updated by others when updating transient log entry") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(_ => true) .transientLog(latest => { // This update will happen first and thus cause version conflict as expected @@ -110,7 +106,7 @@ class FlintTransactionITSuite test("should fail if transient log entry updated by others when updating final log entry") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex) + .startTransaction(testFlintIndex, testMetaLogIndex) .initialLog(_ => true) .transientLog(latest => { diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala index 140046e96..caa33b99d 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -13,8 +13,7 @@ import org.opensearch.flint.spark.skipping.FlintSparkSkippingIndex.getSkippingIn import org.scalatest.matchers.should.Matchers class FlintSparkTransactionITSuite - extends FlintSparkSuite - with OpenSearchTransactionSuite + extends OpenSearchTransactionSuite with Matchers { /** Test table and index name */ From a88c7b77c65ef3c4cee4a9916e11295f22177f17 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 16:25:12 -0700 Subject: [PATCH 18/22] Refactor OS add and update with write method Signed-off-by: Chen Dai --- .../storage/FlintOpenSearchMetadataLog.java | 96 +++++++------------ .../spark/FlintSparkTransactionITSuite.scala | 2 +- 2 files changed, 36 insertions(+), 62 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java index ba9824c4f..43c20de91 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java @@ -5,20 +5,18 @@ package org.opensearch.flint.core.storage; +import static org.opensearch.action.support.WriteRequest.RefreshPolicy; + import java.io.IOException; import java.util.Base64; import java.util.Optional; -import java.util.function.Function; import java.util.logging.Logger; import org.opensearch.OpenSearchException; import org.opensearch.action.DocWriteResponse; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; -import org.opensearch.action.index.IndexResponse; -import org.opensearch.action.support.WriteRequest; import org.opensearch.action.update.UpdateRequest; -import org.opensearch.action.update.UpdateResponse; import org.opensearch.client.RequestOptions; import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.xcontent.XContentType; @@ -94,74 +92,45 @@ public Optional getLatest() { private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { LOG.info("Creating log entry " + logEntry); - try (RestHighLevelClient client = flintClient.createClient()) { - // Assign doc ID here - logEntry = logEntry.copy( - latestId, - logEntry.seqNo(), - logEntry.primaryTerm(), - logEntry.state(), - logEntry.dataSource(), - logEntry.error()); - - IndexResponse response = client.index( - new IndexRequest() - .index(indexName) - .id(logEntry.id()) - .source(logEntry.toJson(), XContentType.JSON), - RequestOptions.DEFAULT); - - // Update seqNo and primaryTerm in log entry - logEntry = logEntry.copy( - logEntry.id(), - response.getSeqNo(), - response.getPrimaryTerm(), - logEntry.state(), - logEntry.dataSource(), - logEntry.error()); - - LOG.info("Log entry created " + logEntry); - return logEntry; - } catch (OpenSearchException | IOException e) { - throw new IllegalStateException("Failed to create initial log entry", e); - } + // Assign doc ID here + FlintMetadataLogEntry logEntryWithId = + logEntry.copy( + latestId, + logEntry.seqNo(), + logEntry.primaryTerm(), + logEntry.state(), + logEntry.dataSource(), + logEntry.error()); + + return writeLogEntry(logEntryWithId, + client -> client.index( + new IndexRequest() + .index(indexName) + .id(logEntryWithId.id()) + .source(logEntryWithId.toJson(), XContentType.JSON), + RequestOptions.DEFAULT)); } private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { LOG.info("Updating log entry " + logEntry); - try (RestHighLevelClient client = flintClient.createClient()) { - UpdateResponse response = - client.update( - new UpdateRequest(indexName, logEntry.id()) - .doc(logEntry.toJson(), XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.WAIT_UNTIL) - .setIfSeqNo(logEntry.seqNo()) - .setIfPrimaryTerm(logEntry.primaryTerm()), - RequestOptions.DEFAULT); - - // Update seqNo and primaryTerm in log entry - logEntry = logEntry.copy( - logEntry.id(), - response.getSeqNo(), - response.getPrimaryTerm(), - logEntry.state(), - logEntry.dataSource(), - logEntry.error()); - - LOG.info("Log entry updated " + logEntry); - return logEntry; - } catch (OpenSearchException | IOException e) { - throw new IllegalStateException("Failed to update log entry: " + logEntry, e); - } + return writeLogEntry(logEntry, + client -> client.update( + new UpdateRequest(indexName, logEntry.id()) + .doc(logEntry.toJson(), XContentType.JSON) + .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) + .setIfSeqNo(logEntry.seqNo()) + .setIfPrimaryTerm(logEntry.primaryTerm()), + RequestOptions.DEFAULT)); } private FlintMetadataLogEntry writeLogEntry( FlintMetadataLogEntry logEntry, - Function write) { + CheckedFunction write) { try (RestHighLevelClient client = flintClient.createClient()) { + // Write (create or update) the doc DocWriteResponse response = write.apply(client); - // Update seqNo and primaryTerm in log entry + // Copy latest seqNo and primaryTerm after write logEntry = logEntry.copy( logEntry.id(), response.getSeqNo(), @@ -176,4 +145,9 @@ private FlintMetadataLogEntry writeLogEntry( throw new IllegalStateException("Failed to write log entry " + logEntry, e); } } + + @FunctionalInterface + public interface CheckedFunction { + R apply(T t) throws IOException; + } } diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala index caa33b99d..b83e8e844 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -17,7 +17,7 @@ class FlintSparkTransactionITSuite with Matchers { /** Test table and index name */ - private val testTable = "spark_catalog.default.test" + private val testTable = "spark_catalog.default.flint_tx_test" private val testFlintIndex = getSkippingIndexName(testTable) private val testLatestId: String = Base64.getEncoder.encodeToString(testFlintIndex.getBytes) From fe76b4ba1f6e4ba798a1f9983f712fd9208f3252 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 16:29:14 -0700 Subject: [PATCH 19/22] Fix broken IT Signed-off-by: Chen Dai --- .../flint/spark/FlintSparkCoveringIndexSqlITSuite.scala | 3 +-- .../flint/spark/FlintSparkSkippingIndexSqlITSuite.scala | 3 +-- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkCoveringIndexSqlITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkCoveringIndexSqlITSuite.scala index b3e2ef063..27419b616 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkCoveringIndexSqlITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkCoveringIndexSqlITSuite.scala @@ -130,8 +130,7 @@ class FlintSparkCoveringIndexSqlITSuite extends FlintSparkSuite { | (name, age) | WITH (auto_refresh = true) | """.stripMargin) - } should have message - "Checkpoint location is mandatory for incremental refresh if spark.flint.index.checkpoint.mandatory enabled" + } } finally { setFlintSparkConf(CHECKPOINT_MANDATORY, "false") } diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkSkippingIndexSqlITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkSkippingIndexSqlITSuite.scala index 7b30349ba..21de15de7 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkSkippingIndexSqlITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkSkippingIndexSqlITSuite.scala @@ -119,8 +119,7 @@ class FlintSparkSkippingIndexSqlITSuite extends FlintSparkSuite { | ( year PARTITION ) | WITH (auto_refresh = true) | """.stripMargin) - } should have message - "Checkpoint location is mandatory for incremental refresh if spark.flint.index.checkpoint.mandatory enabled" + } } finally { setFlintSparkConf(CHECKPOINT_MANDATORY, "false") } From ceb5c4c4e62089cf418b6da726ac12ddb8b0512a Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 17:23:14 -0700 Subject: [PATCH 20/22] Pass datasource name to metadata log Signed-off-by: Chen Dai --- .../opensearch/flint/core/FlintClient.java | 4 +-- .../flint/core/metadata/FlintMetadata.scala | 21 ++++++----- .../log/DefaultOptimisticTransaction.java | 9 ++++- .../core/storage/FlintOpenSearchClient.java | 36 +++++++++++++------ .../storage/FlintOpenSearchMetadataLog.java | 12 +++---- .../opensearch/flint/spark/FlintSpark.scala | 18 +++++----- .../flint/OpenSearchTransactionSuite.scala | 6 ++-- .../flint/core/FlintTransactionITSuite.scala | 12 +++---- 8 files changed, 73 insertions(+), 45 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java index 17358b9a9..c1f5d78c1 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/FlintClient.java @@ -23,10 +23,10 @@ public interface FlintClient { * Start a new optimistic transaction. * * @param indexName index name - * @param metaLogIndexName metadata log index name + * @param dataSourceName TODO: read from elsewhere in future * @return transaction handle */ - OptimisticTransaction startTransaction(String indexName, String metaLogIndexName); + OptimisticTransaction startTransaction(String indexName, String dataSourceName); /** * Create a Flint index with the metadata given. diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/FlintMetadata.scala b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/FlintMetadata.scala index ea0fb0f98..5e2baceab 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/FlintMetadata.scala +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/FlintMetadata.scala @@ -32,6 +32,8 @@ case class FlintMetadata( properties: util.Map[String, AnyRef] = new util.HashMap[String, AnyRef], /** Flint index schema */ schema: util.Map[String, AnyRef] = new util.HashMap[String, AnyRef], + /** Optional latest metadata log entry */ + latestId: Option[String] = None, /** Optional Flint index settings. TODO: move elsewhere? */ indexSettings: Option[String]) { @@ -58,6 +60,9 @@ case class FlintMetadata( .field("source", source) .field("indexedColumns", indexedColumns) + if (latestId.isDefined) { + builder.field("latestId", latestId.get) + } optionalObjectField(builder, "options", options) optionalObjectField(builder, "properties", properties) } @@ -219,14 +224,14 @@ object FlintMetadata { def build(): FlintMetadata = { FlintMetadata( if (version == null) current() else version, - name, - kind, - source, - indexedColumns, - options, - properties, - schema, - indexSettings) + name = name, + kind = kind, + source = source, + indexedColumns = indexedColumns, + options = options, + properties = properties, + schema = schema, + indexSettings = indexSettings) } } } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java index ffb948b2f..2019d8812 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/metadata/log/DefaultOptimisticTransaction.java @@ -24,6 +24,11 @@ public class DefaultOptimisticTransaction implements OptimisticTransaction private static final Logger LOG = Logger.getLogger(DefaultOptimisticTransaction.class.getName()); + /** + * Data source name. TODO: remove this in future. + */ + private final String dataSourceName; + /** * Flint metadata log */ @@ -34,7 +39,9 @@ public class DefaultOptimisticTransaction implements OptimisticTransaction private Function finalAction = null; public DefaultOptimisticTransaction( + String dataSourceName, FlintMetadataLog metadataLog) { + this.dataSourceName = dataSourceName; this.metadataLog = metadataLog; } @@ -95,7 +102,7 @@ private FlintMetadataLogEntry emptyLogEntry() { UNASSIGNED_SEQ_NO, UNASSIGNED_PRIMARY_TERM, IndexState$.MODULE$.EMPTY(), - "mys3", // TODO: get it from spark conf + dataSourceName, ""); } } diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index a456757b7..43f5af85d 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -68,17 +68,26 @@ public class FlintOpenSearchClient implements FlintClient { new NamedXContentRegistry(new SearchModule(Settings.builder().build(), new ArrayList<>()).getNamedXContents()); + /** + * Metadata log index name prefix + */ + public final static String META_LOG_NAME_PREFIX = ".query_execution_request"; + private final FlintOptions options; public FlintOpenSearchClient(FlintOptions options) { this.options = options; } - @Override public OptimisticTransaction startTransaction(String indexName, String metaLogIndexName) { + @Override + public OptimisticTransaction startTransaction(String indexName, String dataSourceName) { + String metaLogIndexName = dataSourceName.isEmpty() ? META_LOG_NAME_PREFIX + : META_LOG_NAME_PREFIX + "_" + dataSourceName; LOG.info("Starting transaction on index " + indexName + " and metadata log index " + metaLogIndexName); + try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { - return new DefaultOptimisticTransaction<>( + return new DefaultOptimisticTransaction<>(dataSourceName, new FlintOpenSearchMetadataLog(this, indexName, metaLogIndexName)); } else { return new NoOptimisticTransaction<>(); @@ -88,7 +97,8 @@ public FlintOpenSearchClient(FlintOptions options) { } } - @Override public void createIndex(String indexName, FlintMetadata metadata) { + @Override + public void createIndex(String indexName, FlintMetadata metadata) { LOG.info("Creating Flint index " + indexName + " with metadata " + metadata); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { @@ -105,7 +115,8 @@ public FlintOpenSearchClient(FlintOptions options) { } } - @Override public boolean exists(String indexName) { + @Override + public boolean exists(String indexName) { LOG.info("Checking if Flint index exists " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { @@ -115,7 +126,8 @@ public FlintOpenSearchClient(FlintOptions options) { } } - @Override public List getAllIndexMetadata(String indexNamePattern) { + @Override + public List getAllIndexMetadata(String indexNamePattern) { LOG.info("Fetching all Flint index metadata for pattern " + indexNamePattern); String osIndexNamePattern = toLowercase(indexNamePattern); try (RestHighLevelClient client = createClient()) { @@ -132,7 +144,8 @@ public FlintOpenSearchClient(FlintOptions options) { } } - @Override public FlintMetadata getIndexMetadata(String indexName) { + @Override + public FlintMetadata getIndexMetadata(String indexName) { LOG.info("Fetching Flint index metadata for " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { @@ -147,7 +160,8 @@ public FlintOpenSearchClient(FlintOptions options) { } } - @Override public void deleteIndex(String indexName) { + @Override + public void deleteIndex(String indexName) { LOG.info("Deleting Flint index " + indexName); String osIndexName = toLowercase(indexName); try (RestHighLevelClient client = createClient()) { @@ -163,10 +177,11 @@ public FlintOpenSearchClient(FlintOptions options) { * Create {@link FlintReader}. * * @param indexName index name. - * @param query DSL query. DSL query is null means match_all. + * @param query DSL query. DSL query is null means match_all. * @return {@link FlintReader}. */ - @Override public FlintReader createReader(String indexName, String query) { + @Override + public FlintReader createReader(String indexName, String query) { LOG.info("Creating Flint index reader for " + indexName + " with query " + query); try { QueryBuilder queryBuilder = new MatchAllQueryBuilder(); @@ -190,7 +205,8 @@ public FlintWriter createWriter(String indexName) { return new OpenSearchWriter(createClient(), toLowercase(indexName), options.getRefreshPolicy()); } - @Override public RestHighLevelClient createClient() { + @Override + public RestHighLevelClient createClient() { RestClientBuilder restClientBuilder = RestClient.builder(new HttpHost(options.getHost(), options.getPort(), options.getScheme())); diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java index 43c20de91..dc2efc595 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchMetadataLog.java @@ -40,16 +40,16 @@ public class FlintOpenSearchMetadataLog implements FlintMetadataLog getLatest() { LOG.info("Fetching latest log entry with id " + latestId); try (RestHighLevelClient client = flintClient.createClient()) { GetResponse response = - client.get(new GetRequest(indexName, latestId), RequestOptions.DEFAULT); + client.get(new GetRequest(metaLogIndexName, latestId), RequestOptions.DEFAULT); if (response.isExists()) { FlintMetadataLogEntry latest = new FlintMetadataLogEntry( @@ -105,7 +105,7 @@ private FlintMetadataLogEntry createLogEntry(FlintMetadataLogEntry logEntry) { return writeLogEntry(logEntryWithId, client -> client.index( new IndexRequest() - .index(indexName) + .index(metaLogIndexName) .id(logEntryWithId.id()) .source(logEntryWithId.toJson(), XContentType.JSON), RequestOptions.DEFAULT)); @@ -115,7 +115,7 @@ private FlintMetadataLogEntry updateLogEntry(FlintMetadataLogEntry logEntry) { LOG.info("Updating log entry " + logEntry); return writeLogEntry(logEntry, client -> client.update( - new UpdateRequest(indexName, logEntry.id()) + new UpdateRequest(metaLogIndexName, logEntry.id()) .doc(logEntry.toJson(), XContentType.JSON) .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) .setIfSeqNo(logEntry.seqNo()) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index 332b9618e..fd93ff26b 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -50,13 +50,11 @@ class FlintSpark(val spark: SparkSession) extends Logging { private val executor: ScheduledExecutorService = Executors.newScheduledThreadPool(1) /** - * Metadata log index name with a default name for backward compatibility. If the index doesn't - * exist, the transaction support will be disabled in FlintClient. + * Data source name. Assign empty string in case of backward compatibility. TODO: remove this in + * future */ - private val metaLogIndexName: String = { - val indexName = spark.conf.getOption("spark.flint.job.requestIndex") - indexName.getOrElse(".query_execution_request") - } + private val dataSourceName: String = + spark.conf.getOption("spark.flint.datasource.name").getOrElse("") /** * Create index builder for creating index with fluent API. @@ -107,7 +105,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { val metadata = index.metadata() try { flintClient - .startTransaction(indexName, metaLogIndexName) + .startTransaction(indexName, dataSourceName) .initialLog(latest => latest.state == EMPTY) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) @@ -137,7 +135,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { try { flintClient - .startTransaction(indexName, metaLogIndexName) + .startTransaction(indexName, dataSourceName) .initialLog(latest => latest.state == ACTIVE) .transientLog(latest => latest.copy(state = REFRESHING)) .finalLog(latest => { @@ -210,7 +208,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { if (flintClient.exists(indexName)) { try { flintClient - .startTransaction(indexName, metaLogIndexName) + .startTransaction(indexName, dataSourceName) .initialLog(latest => latest.state == ACTIVE || latest.state == REFRESHING) .transientLog(latest => latest.copy(state = DELETING)) .finalLog(latest => latest.copy(state = DELETED)) @@ -251,7 +249,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { logInfo("Scheduler triggers index log entry update") try { flintClient - .startTransaction(indexName, metaLogIndexName) + .startTransaction(indexName, dataSourceName) .initialLog(latest => latest.state == REFRESHING) .finalLog(latest => latest) // timestamp will update automatically .commit(latest => logInfo("Updating log entry to " + latest)) diff --git a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala index f5ae10b5e..1e7077799 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/OpenSearchTransactionSuite.scala @@ -18,6 +18,7 @@ import org.opensearch.client.indices.CreateIndexRequest import org.opensearch.common.xcontent.XContentType import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.metadata.log.FlintMetadataLogEntry.IndexState.IndexState +import org.opensearch.flint.core.storage.FlintOpenSearchClient._ import org.opensearch.flint.spark.FlintSparkSuite /** @@ -26,11 +27,12 @@ import org.opensearch.flint.spark.FlintSparkSuite */ trait OpenSearchTransactionSuite extends FlintSparkSuite { - val testMetaLogIndex = ".query_execution_request_mys3" + val testDataSourceName = "myglue" + lazy val testMetaLogIndex: String = META_LOG_NAME_PREFIX + "_" + testDataSourceName override def beforeAll(): Unit = { super.beforeAll() - spark.conf.set("spark.flint.job.requestIndex", testMetaLogIndex) + spark.conf.set("spark.flint.datasource.name", testDataSourceName) } override def beforeEach(): Unit = { diff --git a/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala index 29eae82f4..a6a1dd889 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/core/FlintTransactionITSuite.scala @@ -29,7 +29,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { test("should transit from initial to final log if initial log is empty") { flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(latest => { latest.state shouldBe EMPTY true @@ -43,7 +43,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { test("should transit from initial to final log directly if no transient log") { flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(_ => true) .finalLog(latest => latest.copy(state = ACTIVE)) .commit(_ => latestLogEntry(testLatestId) should contain("state" -> "empty")) @@ -64,7 +64,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { error = "")) flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(latest => { latest.state shouldBe ACTIVE true @@ -79,7 +79,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { test("should exit if initial log entry doesn't meet precondition") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(_ => false) .transientLog(latest => latest) .finalLog(latest => latest) @@ -90,7 +90,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { test("should fail if initial log entry updated by others when updating transient log entry") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(_ => true) .transientLog(latest => { // This update will happen first and thus cause version conflict as expected @@ -106,7 +106,7 @@ class FlintTransactionITSuite extends OpenSearchTransactionSuite with Matchers { test("should fail if transient log entry updated by others when updating final log entry") { the[IllegalStateException] thrownBy { flintClient - .startTransaction(testFlintIndex, testMetaLogIndex) + .startTransaction(testFlintIndex, testDataSourceName) .initialLog(_ => true) .transientLog(latest => { From d9f9e0048e9d96c821213627677223a87db73848 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 17:33:54 -0700 Subject: [PATCH 21/22] Store latest id in Flint metadata Signed-off-by: Chen Dai --- .../opensearch/flint/spark/FlintSpark.scala | 7 ++++++- .../spark/FlintSparkTransactionITSuite.scala | 21 ++++++++++++++++--- 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index fd93ff26b..de08a3a79 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -109,7 +109,12 @@ class FlintSpark(val spark: SparkSession) extends Logging { .initialLog(latest => latest.state == EMPTY) .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) - .commit(latest => flintClient.createIndex(indexName, metadata)) + .commit(latest => + if (latest == null) { + flintClient.createIndex(indexName, metadata) + } else { + flintClient.createIndex(indexName, metadata.copy(latestId = Some(latest.id))) + }) } catch { case e: Exception => logError("Failed to create Flint index", e) diff --git a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala index b83e8e844..5376617dd 100644 --- a/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala +++ b/integ-test/src/test/scala/org/opensearch/flint/spark/FlintSparkTransactionITSuite.scala @@ -7,14 +7,17 @@ package org.opensearch.flint.spark import java.util.Base64 +import org.json4s.{Formats, NoTypeHints} +import org.json4s.native.JsonMethods.parse +import org.json4s.native.Serialization +import org.opensearch.client.RequestOptions +import org.opensearch.client.indices.GetIndexRequest import org.opensearch.flint.OpenSearchTransactionSuite import org.opensearch.flint.spark.FlintSpark.RefreshMode.{FULL, INCREMENTAL} import org.opensearch.flint.spark.skipping.FlintSparkSkippingIndex.getSkippingIndexName import org.scalatest.matchers.should.Matchers -class FlintSparkTransactionITSuite - extends OpenSearchTransactionSuite - with Matchers { +class FlintSparkTransactionITSuite extends OpenSearchTransactionSuite with Matchers { /** Test table and index name */ private val testTable = "spark_catalog.default.flint_tx_test" @@ -37,7 +40,19 @@ class FlintSparkTransactionITSuite .onTable(testTable) .addPartitions("year", "month") .create() + latestLogEntry(testLatestId) should contain("state" -> "active") + + implicit val formats: Formats = Serialization.formats(NoTypeHints) + val mapping = + openSearchClient + .indices() + .get(new GetIndexRequest(testFlintIndex), RequestOptions.DEFAULT) + .getMappings + .get(testFlintIndex) + .source() + .string() + (parse(mapping) \ "_meta" \ "latestId").extract[String] shouldBe testLatestId } test("manual refresh index") { From 8f30429b604066b05aabfce7a44a7373b535a724 Mon Sep 17 00:00:00 2001 From: Chen Dai Date: Mon, 30 Oct 2023 18:41:37 -0700 Subject: [PATCH 22/22] Add comments Signed-off-by: Chen Dai --- .../opensearch/flint/core/storage/FlintOpenSearchClient.java | 4 +++- .../main/scala/org/opensearch/flint/spark/FlintSpark.scala | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java index 43f5af85d..8652f8092 100644 --- a/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java +++ b/flint-core/src/main/scala/org/opensearch/flint/core/storage/FlintOpenSearchClient.java @@ -81,15 +81,17 @@ public FlintOpenSearchClient(FlintOptions options) { @Override public OptimisticTransaction startTransaction(String indexName, String dataSourceName) { + LOG.info("Starting transaction on index " + indexName + " and data source " + dataSourceName); String metaLogIndexName = dataSourceName.isEmpty() ? META_LOG_NAME_PREFIX : META_LOG_NAME_PREFIX + "_" + dataSourceName; - LOG.info("Starting transaction on index " + indexName + " and metadata log index " + metaLogIndexName); try (RestHighLevelClient client = createClient()) { if (client.indices().exists(new GetIndexRequest(metaLogIndexName), RequestOptions.DEFAULT)) { + LOG.info("Found metadata log index " + metaLogIndexName); return new DefaultOptimisticTransaction<>(dataSourceName, new FlintOpenSearchMetadataLog(this, indexName, metaLogIndexName)); } else { + LOG.info("Metadata log index not found " + metaLogIndexName); return new NoOptimisticTransaction<>(); } } catch (IOException e) { diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala index de08a3a79..2713f464a 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/FlintSpark.scala @@ -110,7 +110,7 @@ class FlintSpark(val spark: SparkSession) extends Logging { .transientLog(latest => latest.copy(state = CREATING)) .finalLog(latest => latest.copy(state = ACTIVE)) .commit(latest => - if (latest == null) { + if (latest == null) { // in case transaction capability is disabled flintClient.createIndex(indexName, metadata) } else { flintClient.createIndex(indexName, metadata.copy(latestId = Some(latest.id)))