From 5052ffe03264e12010239ba1dd6a9deab597cfd7 Mon Sep 17 00:00:00 2001 From: Sean Kao Date: Thu, 12 Dec 2024 11:28:40 -0800 Subject: [PATCH 1/3] Add sourceQuery in metadata cache (#988) * add sourceQuery in metadata cache Signed-off-by: Sean Kao * preserve index mapping content when updating cache Signed-off-by: Sean Kao * syntax and comment Signed-off-by: Sean Kao * merge index mapping in place Signed-off-by: Sean Kao --------- Signed-off-by: Sean Kao --- .../metadatacache/FlintMetadataCache.scala | 13 +- .../FlintOpenSearchMetadataCacheWriter.scala | 98 ++++----- .../FlintMetadataCacheSuite.scala | 6 +- ...OpenSearchMetadataCacheWriterITSuite.scala | 196 ++++++------------ 4 files changed, 124 insertions(+), 189 deletions(-) diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCache.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCache.scala index 86267c881..c2007c124 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCache.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCache.scala @@ -23,6 +23,8 @@ case class FlintMetadataCache( refreshInterval: Option[Int], /** Source table names for building the Flint index. */ sourceTables: Array[String], + /** Source query for MV */ + sourceQuery: Option[String], /** Timestamp when Flint index is last refreshed. Unit: milliseconds */ lastRefreshTime: Option[Long]) { @@ -64,6 +66,10 @@ object FlintMetadataCache { case MV_INDEX_TYPE => getSourceTablesFromMetadata(metadata) case _ => Array(metadata.source) } + val sourceQuery = metadata.kind match { + case MV_INDEX_TYPE => Some(metadata.source) + case _ => None + } val lastRefreshTime: Option[Long] = metadata.latestLogEntry.flatMap { entry => entry.lastRefreshCompleteTime match { case FlintMetadataLogEntry.EMPTY_TIMESTAMP => None @@ -71,6 +77,11 @@ object FlintMetadataCache { } } - FlintMetadataCache(metadataCacheVersion, refreshInterval, sourceTables, lastRefreshTime) + FlintMetadataCache( + metadataCacheVersion, + refreshInterval, + sourceTables, + sourceQuery, + lastRefreshTime) } } diff --git a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriter.scala b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriter.scala index f6fc0ba6f..de3e051fb 100644 --- a/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriter.scala +++ b/flint-spark-integration/src/main/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriter.scala @@ -5,18 +5,18 @@ package org.opensearch.flint.spark.metadatacache -import java.util +import java.util.{HashMap, Map => JMap} import scala.collection.JavaConverters._ import org.opensearch.client.RequestOptions +import org.opensearch.client.indices.GetIndexRequest import org.opensearch.client.indices.PutMappingRequest import org.opensearch.common.xcontent.XContentType -import org.opensearch.flint.common.metadata.{FlintIndexMetadataService, FlintMetadata} +import org.opensearch.flint.common.metadata.FlintMetadata import org.opensearch.flint.core.{FlintOptions, IRestHighLevelClient} -import org.opensearch.flint.core.metadata.FlintIndexMetadataServiceBuilder import org.opensearch.flint.core.metadata.FlintJsonHelper._ -import org.opensearch.flint.core.storage.{FlintOpenSearchIndexMetadataService, OpenSearchClientUtils} +import org.opensearch.flint.core.storage.OpenSearchClientUtils import org.apache.spark.internal.Logging @@ -27,27 +27,20 @@ class FlintOpenSearchMetadataCacheWriter(options: FlintOptions) extends FlintMetadataCacheWriter with Logging { - /** - * Since metadata cache shares the index mappings _meta field with OpenSearch index metadata - * storage, this flag is to allow for preserving index metadata that is already stored in _meta - * when updating metadata cache. - */ - private val includeSpec: Boolean = - FlintIndexMetadataServiceBuilder - .build(options) - .isInstanceOf[FlintOpenSearchIndexMetadataService] - override def updateMetadataCache(indexName: String, metadata: FlintMetadata): Unit = { - logInfo(s"Updating metadata cache for $indexName with $metadata"); + logInfo(s"Updating metadata cache for $indexName"); val osIndexName = OpenSearchClientUtils.sanitizeIndexName(indexName) var client: IRestHighLevelClient = null try { client = OpenSearchClientUtils.createClient(options) - val request = new PutMappingRequest(osIndexName) - val serialized = serialize(metadata) - logInfo(s"Serialized: $serialized") - request.source(serialized, XContentType.JSON) - client.updateIndexMapping(request, RequestOptions.DEFAULT) + val indexMapping = getIndexMapping(client, osIndexName) + val metadataCacheProperties = FlintMetadataCache(metadata).toMap.asJava + mergeMetadataCacheProperties(indexMapping, metadataCacheProperties) + val serialized = buildJson(builder => { + builder.field("_meta", indexMapping.get("_meta")) + builder.field("properties", indexMapping.get("properties")) + }) + updateIndexMapping(client, osIndexName, serialized) } catch { case e: Exception => throw new IllegalStateException( @@ -59,50 +52,35 @@ class FlintOpenSearchMetadataCacheWriter(options: FlintOptions) } } - /** - * Serialize FlintMetadataCache from FlintMetadata. Modified from {@link - * FlintOpenSearchIndexMetadataService} - */ - private[metadatacache] def serialize(metadata: FlintMetadata): String = { - try { - buildJson(builder => { - objectField(builder, "_meta") { - // If _meta is used as index metadata storage, preserve them. - if (includeSpec) { - builder - .field("version", metadata.version.version) - .field("name", metadata.name) - .field("kind", metadata.kind) - .field("source", metadata.source) - .field("indexedColumns", metadata.indexedColumns) - - if (metadata.latestId.isDefined) { - builder.field("latestId", metadata.latestId.get) - } - optionalObjectField(builder, "options", metadata.options) - } - - optionalObjectField(builder, "properties", buildPropertiesMap(metadata)) - } - builder.field("properties", metadata.schema) - }) - } catch { - case e: Exception => - throw new IllegalStateException("Failed to jsonify cache metadata", e) - } + private[metadatacache] def getIndexMapping( + client: IRestHighLevelClient, + osIndexName: String): JMap[String, AnyRef] = { + val request = new GetIndexRequest(osIndexName) + val response = client.getIndex(request, RequestOptions.DEFAULT) + response.getMappings.get(osIndexName).sourceAsMap() } /** - * Since _meta.properties is shared by both index metadata and metadata cache, here we merge the - * two maps. + * Merge metadata cache properties into index mapping in place. Metadata cache is written into + * _meta.properties field of index mapping. */ - private def buildPropertiesMap(metadata: FlintMetadata): util.Map[String, AnyRef] = { - val metadataCacheProperties = FlintMetadataCache(metadata).toMap + private def mergeMetadataCacheProperties( + indexMapping: JMap[String, AnyRef], + metadataCacheProperties: JMap[String, AnyRef]): Unit = { + indexMapping + .computeIfAbsent("_meta", _ => new HashMap[String, AnyRef]()) + .asInstanceOf[JMap[String, AnyRef]] + .computeIfAbsent("properties", _ => new HashMap[String, AnyRef]()) + .asInstanceOf[JMap[String, AnyRef]] + .putAll(metadataCacheProperties) + } - if (includeSpec) { - (metadataCacheProperties ++ metadata.properties.asScala).asJava - } else { - metadataCacheProperties.asJava - } + private[metadatacache] def updateIndexMapping( + client: IRestHighLevelClient, + osIndexName: String, + mappingSource: String): Unit = { + val request = new PutMappingRequest(osIndexName) + request.source(mappingSource, XContentType.JSON) + client.updateIndexMapping(request, RequestOptions.DEFAULT) } } diff --git a/flint-spark-integration/src/test/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCacheSuite.scala b/flint-spark-integration/src/test/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCacheSuite.scala index 6ec6cf696..971c18857 100644 --- a/flint-spark-integration/src/test/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCacheSuite.scala +++ b/flint-spark-integration/src/test/scala/org/opensearch/flint/spark/metadatacache/FlintMetadataCacheSuite.scala @@ -83,11 +83,13 @@ class FlintMetadataCacheSuite extends AnyFlatSpec with Matchers { } it should "construct from materialized view FlintMetadata" in { + val testQuery = + "SELECT 1 FROM spark_catalog.default.test_table UNION SELECT 1 FROM spark_catalog.default.another_table" val content = s""" { | "_meta": { | "kind": "$MV_INDEX_TYPE", - | "source": "spark_catalog.default.wrong_table", + | "source": "$testQuery", | "options": { | "auto_refresh": "true", | "refresh_interval": "10 Minutes" @@ -116,6 +118,7 @@ class FlintMetadataCacheSuite extends AnyFlatSpec with Matchers { metadataCache.sourceTables shouldBe Array( "spark_catalog.default.test_table", "spark_catalog.default.another_table") + metadataCache.sourceQuery.get shouldBe testQuery metadataCache.lastRefreshTime.get shouldBe 1234567890123L } @@ -145,6 +148,7 @@ class FlintMetadataCacheSuite extends AnyFlatSpec with Matchers { metadataCache.metadataCacheVersion shouldBe FlintMetadataCache.metadataCacheVersion metadataCache.refreshInterval shouldBe empty metadataCache.sourceTables shouldBe Array("spark_catalog.default.test_table") + metadataCache.sourceQuery shouldBe empty metadataCache.lastRefreshTime shouldBe empty } } diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriterITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriterITSuite.scala index 5b4dd0208..692a0c2ff 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriterITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/metadatacache/FlintOpenSearchMetadataCacheWriterITSuite.scala @@ -5,18 +5,17 @@ package org.opensearch.flint.spark.metadatacache -import java.util.{Base64, List} +import java.util.{Base64, Map => JMap} import scala.collection.JavaConverters._ import com.stephenn.scalatest.jsonassert.JsonMatchers.matchJson +import org.json4s.{DefaultFormats, Extraction, JValue} import org.json4s.native.JsonMethods._ -import org.opensearch.flint.common.FlintVersion.current -import org.opensearch.flint.common.metadata.FlintMetadata import org.opensearch.flint.common.metadata.log.FlintMetadataLogEntry import org.opensearch.flint.core.FlintOptions -import org.opensearch.flint.core.storage.{FlintOpenSearchClient, FlintOpenSearchIndexMetadataService} -import org.opensearch.flint.spark.{FlintSparkIndexOptions, FlintSparkSuite} +import org.opensearch.flint.core.storage.{FlintOpenSearchClient, FlintOpenSearchIndexMetadataService, OpenSearchClientUtils} +import org.opensearch.flint.spark.{FlintSpark, FlintSparkIndexOptions, FlintSparkSuite} import org.opensearch.flint.spark.covering.FlintSparkCoveringIndex.COVERING_INDEX_TYPE import org.opensearch.flint.spark.mv.FlintSparkMaterializedView import org.opensearch.flint.spark.mv.FlintSparkMaterializedView.MV_INDEX_TYPE @@ -69,63 +68,24 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat } test("build opensearch metadata cache writer") { - setFlintSparkConf(FlintSparkConf.METADATA_CACHE_WRITE, "true") withMetadataCacheWriteEnabled { FlintMetadataCacheWriterBuilder .build(FlintSparkConf()) shouldBe a[FlintOpenSearchMetadataCacheWriter] } } - test("serialize metadata cache to JSON") { - val expectedMetadataJson: String = s""" - | { - | "_meta": { - | "version": "${current()}", - | "name": "$testFlintIndex", - | "kind": "test_kind", - | "source": "$testTable", - | "indexedColumns": [ - | { - | "test_field": "spark_type" - | }], - | "options": { - | "auto_refresh": "true", - | "refresh_interval": "10 Minutes" - | }, - | "properties": { - | "metadataCacheVersion": "${FlintMetadataCache.metadataCacheVersion}", - | "refreshInterval": 600, - | "sourceTables": ["$testTable"], - | "lastRefreshTime": $testLastRefreshCompleteTime - | }, - | "latestId": "$testLatestId" - | }, - | "properties": { - | "test_field": { - | "type": "os_type" - | } - | } - | } - |""".stripMargin - val builder = new FlintMetadata.Builder - builder.name(testFlintIndex) - builder.kind("test_kind") - builder.source(testTable) - builder.addIndexedColumn(Map[String, AnyRef]("test_field" -> "spark_type").asJava) - builder.options( - Map("auto_refresh" -> "true", "refresh_interval" -> "10 Minutes") - .mapValues(_.asInstanceOf[AnyRef]) - .asJava) - builder.schema(Map[String, AnyRef]("test_field" -> Map("type" -> "os_type").asJava).asJava) - builder.latestLogEntry(flintMetadataLogEntry) - - val metadata = builder.build() - flintMetadataCacheWriter.serialize(metadata) should matchJson(expectedMetadataJson) - } - test("write metadata cache to index mappings") { + val content = + s""" { + | "properties": { + | "age": { + | "type": "integer" + | } + | } + | } + |""".stripMargin val metadata = FlintOpenSearchIndexMetadataService - .deserialize("{}") + .deserialize(content) .copy(latestLogEntry = Some(flintMetadataLogEntry)) flintClient.createIndex(testFlintIndex, metadata) flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, metadata) @@ -139,7 +99,7 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat } Seq(SKIPPING_INDEX_TYPE, COVERING_INDEX_TYPE).foreach { case kind => - test(s"write metadata cache to $kind index mappings with source tables") { + test(s"write metadata cache to $kind index mappings with source tables for non mv index") { val content = s""" { | "_meta": { @@ -164,10 +124,11 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat .get("sourceTables") .asInstanceOf[java.util.ArrayList[String]] should contain theSameElementsAs Array( testTable) + properties should not contain key("sourceQuery") } } - test("write metadata cache with source tables from index metadata") { + test("write metadata cache with source tables and query from mv index metadata") { val mv = FlintSparkMaterializedView( "spark_catalog.default.mv", s"SELECT 1 FROM $testTable", @@ -182,9 +143,12 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat properties .get("sourceTables") .asInstanceOf[java.util.ArrayList[String]] should contain theSameElementsAs Array(testTable) + properties + .get("sourceQuery") + .asInstanceOf[String] shouldBe s"SELECT 1 FROM $testTable" } - test("write metadata cache with source tables from deserialized metadata") { + test("write metadata cache with source tables and query from deserialized mv metadata") { val testTable2 = "spark_catalog.default.metadatacache_test2" val content = s""" { @@ -272,31 +236,39 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, metadata) val properties = flintIndexMetadataService.getIndexMetadata(testFlintIndex).properties - properties should have size 3 - properties should contain allOf (Entry( - "metadataCacheVersion", - FlintMetadataCache.metadataCacheVersion), - Entry("lastRefreshTime", testLastRefreshCompleteTime)) + properties should not contain key("refreshInterval") } test("exclude last refresh time in metadata cache when index has not been refreshed") { + val content = + s""" { + | "properties": { + | "age": { + | "type": "integer" + | } + | } + | } + |""".stripMargin val metadata = FlintOpenSearchIndexMetadataService - .deserialize("{}") + .deserialize(content) .copy(latestLogEntry = Some(flintMetadataLogEntry.copy(lastRefreshCompleteTime = 0L))) flintClient.createIndex(testFlintIndex, metadata) flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, metadata) val properties = flintIndexMetadataService.getIndexMetadata(testFlintIndex).properties - properties should have size 2 - properties should contain( - Entry("metadataCacheVersion", FlintMetadataCache.metadataCacheVersion)) + properties should not contain key("lastRefreshTime") } test("write metadata cache to index mappings and preserve other index metadata") { val content = """ { | "_meta": { - | "kind": "test_kind" + | "kind": "test_kind", + | "name": "test_name", + | "custom": "test_custom", + | "properties": { + | "custom_in_properties": "test_custom" + | } | }, | "properties": { | "age": { @@ -311,48 +283,31 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat .copy(latestLogEntry = Some(flintMetadataLogEntry)) flintClient.createIndex(testFlintIndex, metadata) - flintIndexMetadataService.updateIndexMetadata(testFlintIndex, metadata) - flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, metadata) + // Simulates index mapping updated by custom implementation of FlintIndexMetadataService + // with the extra "custom" field. + val client = OpenSearchClientUtils.createClient(options) + flintMetadataCacheWriter.updateIndexMapping(client, testFlintIndex, content) - flintIndexMetadataService.getIndexMetadata(testFlintIndex).kind shouldBe "test_kind" - flintIndexMetadataService.getIndexMetadata(testFlintIndex).name shouldBe empty - flintIndexMetadataService.getIndexMetadata(testFlintIndex).schema should have size 1 - var properties = flintIndexMetadataService.getIndexMetadata(testFlintIndex).properties - properties should have size 3 - properties should contain allOf (Entry( - "metadataCacheVersion", - FlintMetadataCache.metadataCacheVersion), - Entry("lastRefreshTime", testLastRefreshCompleteTime)) - - val newContent = - """ { - | "_meta": { - | "kind": "test_kind", - | "name": "test_name" - | }, - | "properties": { - | "age": { - | "type": "integer" - | } - | } - | } - |""".stripMargin - - val newMetadata = FlintOpenSearchIndexMetadataService - .deserialize(newContent) - .copy(latestLogEntry = Some(flintMetadataLogEntry)) - flintIndexMetadataService.updateIndexMetadata(testFlintIndex, newMetadata) - flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, newMetadata) + flintMetadataCacheWriter.updateMetadataCache(testFlintIndex, metadata) flintIndexMetadataService.getIndexMetadata(testFlintIndex).kind shouldBe "test_kind" flintIndexMetadataService.getIndexMetadata(testFlintIndex).name shouldBe "test_name" flintIndexMetadataService.getIndexMetadata(testFlintIndex).schema should have size 1 - properties = flintIndexMetadataService.getIndexMetadata(testFlintIndex).properties - properties should have size 3 + val properties = flintIndexMetadataService.getIndexMetadata(testFlintIndex).properties + properties should have size 4 properties should contain allOf (Entry( "metadataCacheVersion", FlintMetadataCache.metadataCacheVersion), - Entry("lastRefreshTime", testLastRefreshCompleteTime)) + Entry("lastRefreshTime", testLastRefreshCompleteTime), Entry( + "custom_in_properties", + "test_custom")) + + // Directly get the index mapping and verify custom field is preserved + flintMetadataCacheWriter + .getIndexMapping(client, testFlintIndex) + .get("_meta") + .asInstanceOf[JMap[String, AnyRef]] + .get("custom") shouldBe "test_custom" } Seq( @@ -391,6 +346,7 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat test(s"write metadata cache for $refreshMode") { withExternalSchedulerEnabled { withMetadataCacheWriteEnabled { + val flint: FlintSpark = new FlintSpark(spark) withTempDir { checkpointDir => // update checkpoint_location if available in optionsMap val indexOptions = FlintSparkIndexOptions( @@ -407,25 +363,12 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat .options(indexOptions, testFlintIndex) .create() - var index = flint.describeIndex(testFlintIndex) - index shouldBe defined - val propertiesJson = - compact( - render( - parse( - flintMetadataCacheWriter.serialize( - index.get.metadata())) \ "_meta" \ "properties")) + val propertiesJson = compact(render(getPropertiesJValue(testFlintIndex))) propertiesJson should matchJson(expectedJson) flint.refreshIndex(testFlintIndex) - index = flint.describeIndex(testFlintIndex) - index shouldBe defined val lastRefreshTime = - compact( - render( - parse( - flintMetadataCacheWriter.serialize( - index.get.metadata())) \ "_meta" \ "properties" \ "lastRefreshTime")).toLong + compact(render(getPropertiesJValue(testFlintIndex) \ "lastRefreshTime")).toLong lastRefreshTime should be > 0L } } @@ -435,6 +378,7 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat test("write metadata cache for auto refresh index with internal scheduler") { withMetadataCacheWriteEnabled { + val flint: FlintSpark = new FlintSpark(spark) withTempDir { checkpointDir => flint .skippingIndex() @@ -450,12 +394,7 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat testFlintIndex) .create() - var index = flint.describeIndex(testFlintIndex) - index shouldBe defined - val propertiesJson = - compact( - render(parse( - flintMetadataCacheWriter.serialize(index.get.metadata())) \ "_meta" \ "properties")) + val propertiesJson = compact(render(getPropertiesJValue(testFlintIndex))) propertiesJson should matchJson(s""" | { | "metadataCacheVersion": "${FlintMetadataCache.metadataCacheVersion}", @@ -465,12 +404,15 @@ class FlintOpenSearchMetadataCacheWriterITSuite extends FlintSparkSuite with Mat |""".stripMargin) flint.refreshIndex(testFlintIndex) - index = flint.describeIndex(testFlintIndex) - index shouldBe defined - compact(render(parse( - flintMetadataCacheWriter.serialize( - index.get.metadata())) \ "_meta" \ "properties")) should not include "lastRefreshTime" + compact(render(getPropertiesJValue(testFlintIndex))) should not include "lastRefreshTime" } } } + + private def getPropertiesJValue(indexName: String): JValue = { + // Convert to scala map because json4s converts java.util.Map into an empty JObject + // https://github.com/json4s/json4s/issues/392 + val properties = flintIndexMetadataService.getIndexMetadata(indexName).properties.asScala + Extraction.decompose(properties)(DefaultFormats) + } } From d78984835c7692ac342e1e0732513ffa93f6b2a0 Mon Sep 17 00:00:00 2001 From: YANGDB Date: Mon, 16 Dec 2024 10:00:39 -0800 Subject: [PATCH 2/3] PPL support json_delete, append functions (#971) * add the following JSON functions - json_delete - json_append - json_extend Signed-off-by: YANGDB * add the following JSON functions & test for - json_delete - json_append - json_extend Signed-off-by: YANGDB * update tests & scala-fmt Signed-off-by: YANGDB * update IT tests for `json_delete` Signed-off-by: YANGDB * update IT tests for `json_delete` Signed-off-by: YANGDB * update IT tests for `json_append` Signed-off-by: YANGDB * update IT tests according to PR's comments Signed-off-by: YANGDB * update PR to only inclide json_delete json_append Signed-off-by: YANGDB * update PR to only inclide json_delete json_append Signed-off-by: YANGDB * fix scalaFmt Signed-off-by: YANGDB * fix tests Signed-off-by: YANGDB * fix documentation Signed-off-by: YANGDB --------- Signed-off-by: YANGDB --- docs/ppl-lang/functions/ppl-json.md | 84 ++++++ .../FlintSparkPPLJsonFunctionITSuite.scala | 283 ++++++++++++++++++ .../src/main/antlr4/OpenSearchPPLLexer.g4 | 6 +- .../src/main/antlr4/OpenSearchPPLParser.g4 | 4 +- .../sql/expression/function/JsonUtils.java | 106 +++++++ .../expression/function/SerializableUdf.java | 141 ++++++++- .../sql/ppl/CatalystExpressionVisitor.java | 24 +- .../ppl/utils/BuiltinFunctionTransformer.java | 16 +- ...dfTest.java => SerializableIPUdfTest.java} | 13 +- .../function/SerializableJsonUdfTest.java | 153 ++++++++++ ...PlanJsonFunctionsTranslatorTestSuite.scala | 53 +++- ...PLLogicalPlanParseCidrmatchTestSuite.scala | 43 +-- 12 files changed, 847 insertions(+), 79 deletions(-) create mode 100644 ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/JsonUtils.java rename ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/{SerializableUdfTest.java => SerializableIPUdfTest.java} (87%) create mode 100644 ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableJsonUdfTest.java diff --git a/docs/ppl-lang/functions/ppl-json.md b/docs/ppl-lang/functions/ppl-json.md index 2c0c0ca67..3eb952cb7 100644 --- a/docs/ppl-lang/functions/ppl-json.md +++ b/docs/ppl-lang/functions/ppl-json.md @@ -203,6 +203,90 @@ Example: +----------------+ +### `JSON_DELETE` + +**Description** + +`json_delete(json_string, [keys list])` Deletes json elements from a json object based on json specific keys. Return the updated object after keys deletion . + +**Arguments type:** JSON_STRING, List + +**Return type:** JSON_STRING + +A JSON object format. + +Example: + + os> source=people | eval deleted = json_delete('{"account_number":1,"balance":39225,"age":32,"gender":"M"}', array('age','gender')) | head 1 | fields deleted + fetched rows / total rows = 1/1 + +------------------------------------------+ + | deleted | + +-----------------------------------------+ + |{"account_number":1,"balance":39225} | + +-----------------------------------------+ + + os> source=people | eval deleted = json_delete('{"f1":"abc","f2":{"f3":"a","f4":"b"}}', array('f2.f3')) | head 1 | fields deleted + fetched rows / total rows = 1/1 + +-----------------------------------------------------------+ + | deleted | + +-----------------------------------------------------------+ + | {"f1":"abc","f2":{"f4":"b"}} | + +-----------------------------------------------------------+ + + os> source=people | eval deleted = json_delete('{"teacher":"Alice","student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]}',array('teacher', 'student.rank')) | head 1 | fields deleted + fetched rows / total rows = 1/1 + +--------------------------------------------------+ + | deleted | + +--------------------------------------------------+ + |{"student":[{"name":"Bob"},{"name":"Charlie"}]} | + +--------------------------------------------------+ + +### `JSON_APPEND` + +**Description** + +`json_append(json_string, [path_key, list of values to add ])` appends values to end of an array within the json elements. Return the updated json object after appending . + +**Argument type:** JSON_STRING, List + +**Return type:** JSON_STRING + +A string JSON object format. + +**Note** +Append adds the value to the end of the existing array with the following cases: + - path is an object value - append is ignored and the value is returned + - path is an existing array not empty - the value are added to the array's tail + - path not found - the value are added to the root of the json tree + - path is an existing array is empty - create a new array with the given value + +Example: + + os> source=people | eval append = json_append(`{"teacher":["Alice"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]}`,array('student', '{"name":"Tomy","rank":5}')) | head 1 | fields append + fetched rows / total rows = 1/1 + +-----------------------------------------------------------------------------------------------------------------------------------+ + | append | + +-----------------------------------------------------------------------------------------------------------------------------------+ + |{"teacher":["Alice"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2},{"name":"Tomy","rank":5}]} | + +-----------------------------------------------------------------------------------------------------------------------------------+ + + os> source=people | eval append = json_append(`{"teacher":["Alice"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]}`,array('teacher', 'Tom', 'Walt')) | head 1 | fields append + fetched rows / total rows = 1/1 + +-----------------------------------------------------------------------------------------------------------------------------------+ + | append | + +-----------------------------------------------------------------------------------------------------------------------------------+ + |{"teacher":["Alice","Tom","Walt"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]} | + +-----------------------------------------------------------------------------------------------------------------------------------+ + + + os> source=people | eval append = json_append(`{"school":{"teacher":["Alice"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]}}`,array('school.teacher', 'Tom', 'Walt')) | head 1 | fields append + fetched rows / total rows = 1/1 + +-------------------------------------------------------------------------------------------------------------------------+ + | append | + +-------------------------------------------------------------------------------------------------------------------------+ + |{"school":{"teacher":["Alice","Tom","Walt"],"student":[{"name":"Bob","rank":1},{"name":"Charlie","rank":2}]}} | + +-------------------------------------------------------------------------------------------------------------------------+ + ### `JSON_KEYS` **Description** diff --git a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLJsonFunctionITSuite.scala b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLJsonFunctionITSuite.scala index fca758101..7a00d9a07 100644 --- a/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLJsonFunctionITSuite.scala +++ b/integ-test/src/integration/scala/org/opensearch/flint/spark/ppl/FlintSparkPPLJsonFunctionITSuite.scala @@ -5,6 +5,10 @@ package org.opensearch.flint.spark.ppl +import java.util + +import org.opensearch.sql.expression.function.SerializableUdf.visit + import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.expressions.{Alias, EqualTo, Literal, Not} @@ -27,6 +31,11 @@ class FlintSparkPPLJsonFunctionITSuite private val validJson5 = "{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}" private val validJson6 = "[1,2,3]" + private val validJson7 = + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}" + private val validJson8 = + "{\"school\":{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}}" + private val validJson9 = "{\"a\":[\"valueA\", \"valueB\"]}" private val invalidJson1 = "[1,2" private val invalidJson2 = "[invalid json]" private val invalidJson3 = "{\"invalid\": \"json\"" @@ -385,4 +394,278 @@ class FlintSparkPPLJsonFunctionITSuite null)) assertSameRows(expectedSeq, frame) } + + test("test json_delete() function: one key") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_delete('$validJson1',array('age')) | head 1 | fields result + | """.stripMargin) + assertSameRows(Seq(Row("{\"account_number\":1,\"balance\":39225,\"gender\":\"M\"}")), frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = UnresolvedFunction("array", Seq(Literal("age")), isDistinct = false) + val jsonObjExp = + Literal("{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_delete() function: multiple keys") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_delete('$validJson1',array('age','gender')) | head 1 | fields result + | """.stripMargin) + assertSameRows(Seq(Row("{\"account_number\":1,\"balance\":39225}")), frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("age"), Literal("gender")), isDistinct = false) + val jsonObjExp = + Literal("{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_delete() function: nested key") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_delete('$validJson2',array('f2.f3')) | head 1 | fields result + | """.stripMargin) + assertSameRows(Seq(Row("{\"f1\":\"abc\",\"f2\":{\"f4\":\"b\"}}")), frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("f2.f3")), isDistinct = false) + val jsonObjExp = + Literal("{\"f1\":\"abc\",\"f2\":{\"f3\":\"a\",\"f4\":\"b\"}}") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_delete() function: multi depth keys ") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_delete('$validJson5',array('teacher', 'student.rank')) | head 1 | fields result + | """.stripMargin) + assertSameRows(Seq(Row("{\"student\":[{\"name\":\"Bob\"},{\"name\":\"Charlie\"}]}")), frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("teacher"), Literal("student.rank")), + isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_delete() function: key not found") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_delete('$validJson5',array('none')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("none")), isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add single value") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson7',array('teacher', 'Tom')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":[\"Alice\",\"Tom\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("teacher"), Literal("Tom")), isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add single value key not found") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson7',array('headmaster', 'Tom')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}],\"headmaster\":[\"Tom\"]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("headmaster"), Literal("Tom")), isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add single Object key not found") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson7',array('headmaster', '{"name":"Tomy","rank":1}')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}],\"headmaster\":[{\"name\":\"Tomy\",\"rank\":1}]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("headmaster"), Literal("""{"name":"Tomy","rank":1}""")), + isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add single Object value") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson7',array('student', '{"name":"Tomy","rank":5}')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2},{\"name\":\"Tomy\",\"rank\":5}]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("student"), Literal("""{"name":"Tomy","rank":5}""")), + isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add multi value") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson7',array('teacher', 'Tom', 'Walt')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"teacher\":[\"Alice\",\"Tom\",\"Walt\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("teacher"), Literal("Tom"), Literal("Walt")), + isDistinct = false) + val jsonObjExp = + Literal( + "{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } + + test("test json_append() function: add nested value") { + val frame = sql(s""" + | source = $testTable + | | eval result = json_append('$validJson8',array('school.teacher', 'Tom', 'Walt')) | head 1 | fields result + | """.stripMargin) + assertSameRows( + Seq(Row( + "{\"school\":{\"teacher\":[\"Alice\",\"Tom\",\"Walt\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}}")), + frame) + + val logicalPlan: LogicalPlan = frame.queryExecution.logical + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("school.teacher"), Literal("Tom"), Literal("Walt")), + isDistinct = false) + val jsonObjExp = + Literal( + "{\"school\":{\"teacher\":[\"Alice\"],\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}}") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val limit = GlobalLimit(Literal(1), LocalLimit(Literal(1), eval)) + val expectedPlan = Project(Seq(UnresolvedAttribute("result")), limit) + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) + } } diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 index d15f5c8e3..b7d615980 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLLexer.g4 @@ -385,11 +385,11 @@ JSON_ARRAY: 'JSON_ARRAY'; JSON_ARRAY_LENGTH: 'JSON_ARRAY_LENGTH'; TO_JSON_STRING: 'TO_JSON_STRING'; JSON_EXTRACT: 'JSON_EXTRACT'; +JSON_DELETE : 'JSON_DELETE'; JSON_KEYS: 'JSON_KEYS'; JSON_VALID: 'JSON_VALID'; -//JSON_APPEND: 'JSON_APPEND'; -//JSON_DELETE: 'JSON_DELETE'; -//JSON_EXTEND: 'JSON_EXTEND'; +JSON_APPEND: 'JSON_APPEND'; +//JSON_EXTEND : 'JSON_EXTEND'; //JSON_SET: 'JSON_SET'; //JSON_ARRAY_ALL_MATCH: 'JSON_ARRAY_ALL_MATCH'; //JSON_ARRAY_ANY_MATCH: 'JSON_ARRAY_ANY_MATCH'; diff --git a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 index 2466a3d23..b990fd549 100644 --- a/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 +++ b/ppl-spark-integration/src/main/antlr4/OpenSearchPPLParser.g4 @@ -875,10 +875,10 @@ jsonFunctionName | JSON_ARRAY_LENGTH | TO_JSON_STRING | JSON_EXTRACT + | JSON_DELETE + | JSON_APPEND | JSON_KEYS | JSON_VALID -// | JSON_APPEND -// | JSON_DELETE // | JSON_EXTEND // | JSON_SET // | JSON_ARRAY_ALL_MATCH diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/JsonUtils.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/JsonUtils.java new file mode 100644 index 000000000..9ca6732c6 --- /dev/null +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/JsonUtils.java @@ -0,0 +1,106 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.sql.expression.function; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +public interface JsonUtils { + ObjectMapper objectMapper = new ObjectMapper(); + + static Object parseValue(String value) { + // Try parsing the value as JSON, fallback to primitive if parsing fails + try { + return objectMapper.readValue(value, Object.class); + } catch (Exception e) { + // Primitive value, return as is + return value; + } + } + + /** + * append nested value to the json object + * @param currentObj + * @param pathParts + * @param depth + * @param valueToAppend + */ + static void appendNestedValue(Object currentObj, String[] pathParts, int depth, Object valueToAppend) { + if (currentObj == null || depth >= pathParts.length) { + return; + } + + if (currentObj instanceof Map) { + Map currentMap = (Map) currentObj; + String currentKey = pathParts[depth]; + + if (depth == pathParts.length - 1) { + // If it's the last key, append to the array + currentMap.computeIfAbsent(currentKey, k -> new ArrayList<>()); // Create list if not present + Object existingValue = currentMap.get(currentKey); + + if (existingValue instanceof List) { + List existingList = (List) existingValue; + existingList.add(valueToAppend); + } + } else { + // Continue traversing + currentMap.computeIfAbsent(currentKey, k -> new LinkedHashMap<>()); // Create map if not present + appendNestedValue(currentMap.get(currentKey), pathParts, depth + 1, valueToAppend); + } + } else if (currentObj instanceof List) { + // If the current object is a list, process each map in the list + List list = (List) currentObj; + for (Object item : list) { + if (item instanceof Map) { + appendNestedValue(item, pathParts, depth, valueToAppend); + } + } + } + } + + /** + * remove nested json object using its keys parts + * @param currentObj + * @param keyParts + * @param depth + */ + static void removeNestedKey(Object currentObj, String[] keyParts, int depth) { + if (currentObj == null || depth >= keyParts.length) { + return; + } + + if (currentObj instanceof Map) { + Map currentMap = (Map) currentObj; + String currentKey = keyParts[depth]; + + if (depth == keyParts.length - 1) { + // If it's the last key, remove it from the map + currentMap.remove(currentKey); + } else { + // If not the last key, continue traversing + if (currentMap.containsKey(currentKey)) { + Object nextObj = currentMap.get(currentKey); + + if (nextObj instanceof List) { + // If the value is a list, process each item in the list + List list = (List) nextObj; + for (int i = 0; i < list.size(); i++) { + removeNestedKey(list.get(i), keyParts, depth + 1); + } + } else { + // Continue traversing if it's a map + removeNestedKey(nextObj, keyParts, depth + 1); + } + } + } + } + } +} diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/SerializableUdf.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/SerializableUdf.java index 2541b3743..e80a26bc4 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/SerializableUdf.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/expression/function/SerializableUdf.java @@ -8,14 +8,105 @@ import inet.ipaddr.AddressStringException; import inet.ipaddr.IPAddressString; import inet.ipaddr.IPAddressStringParameters; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.ScalaUDF; +import org.apache.spark.sql.types.DataTypes; import scala.Function2; +import scala.Option; import scala.Serializable; +import scala.collection.JavaConverters; +import scala.collection.mutable.WrappedArray; import scala.runtime.AbstractFunction2; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +import static org.opensearch.sql.expression.function.JsonUtils.appendNestedValue; +import static org.opensearch.sql.expression.function.JsonUtils.objectMapper; +import static org.opensearch.sql.expression.function.JsonUtils.parseValue; +import static org.opensearch.sql.expression.function.JsonUtils.removeNestedKey; +import static org.opensearch.sql.ppl.utils.DataTypeTransformer.seq; + public interface SerializableUdf { - Function2 cidrFunction = new SerializableAbstractFunction2<>() { + + abstract class SerializableAbstractFunction2 extends AbstractFunction2 + implements Serializable { + } + + /** + * Remove specified keys from a JSON string. + * + * @param jsonStr The input JSON string. + * @param keysToRemove The list of keys to remove. + * @return A new JSON string without the specified keys. + */ + Function2, String> jsonDeleteFunction = new SerializableAbstractFunction2<>() { + @Override + public String apply(String jsonStr, WrappedArray keysToRemove) { + if (jsonStr == null) { + return null; + } + try { + Map jsonMap = objectMapper.readValue(jsonStr, Map.class); + removeKeys(jsonMap, keysToRemove); + return objectMapper.writeValueAsString(jsonMap); + } catch (Exception e) { + return null; + } + } + + private void removeKeys(Map map, WrappedArray keysToRemove) { + Collection keys = JavaConverters.asJavaCollection(keysToRemove); + for (String key : keys) { + String[] keyParts = key.split("\\."); + removeNestedKey(map, keyParts, 0); + } + } + }; + + Function2, String> jsonAppendFunction = new SerializableAbstractFunction2<>() { + /** + * Append values to JSON arrays based on specified path-values. + * + * @param jsonStr The input JSON string. + * @param elements A list of path-values where the first item is the path and subsequent items are values to append. + * @return The updated JSON string. + */ + public String apply(String jsonStr, WrappedArray elements) { + if (jsonStr == null) { + return null; + } + try { + List pathValues = JavaConverters.mutableSeqAsJavaList(elements); + if (pathValues.isEmpty()) { + return jsonStr; + } + + String path = pathValues.get(0); + String[] pathParts = path.split("\\."); + List values = pathValues.subList(1, pathValues.size()); + + // Parse the JSON string into a Map + Map jsonMap = objectMapper.readValue(jsonStr, Map.class); + + // Append each value at the specified path + for (String value : values) { + Object parsedValue = parseValue(value); // Parse the value + appendNestedValue(jsonMap, pathParts, 0, parsedValue); + } + + // Convert the updated map back to JSON + return objectMapper.writeValueAsString(jsonMap); + } catch (Exception e) { + return null; + } + } + }; + + Function2 cidrFunction = new SerializableAbstractFunction2<>() { IPAddressStringParameters valOptions = new IPAddressStringParameters.Builder() .allowEmpty(false) @@ -32,7 +123,7 @@ public Boolean apply(String ipAddress, String cidrBlock) { try { parsedIpAddress.validate(); } catch (AddressStringException e) { - throw new RuntimeException("The given ipAddress '"+ipAddress+"' is invalid. It must be a valid IPv4 or IPv6 address. Error details: "+e.getMessage()); + throw new RuntimeException("The given ipAddress '" + ipAddress + "' is invalid. It must be a valid IPv4 or IPv6 address. Error details: " + e.getMessage()); } IPAddressString parsedCidrBlock = new IPAddressString(cidrBlock, valOptions); @@ -40,18 +131,54 @@ public Boolean apply(String ipAddress, String cidrBlock) { try { parsedCidrBlock.validate(); } catch (AddressStringException e) { - throw new RuntimeException("The given cidrBlock '"+cidrBlock+"' is invalid. It must be a valid CIDR or netmask. Error details: "+e.getMessage()); + throw new RuntimeException("The given cidrBlock '" + cidrBlock + "' is invalid. It must be a valid CIDR or netmask. Error details: " + e.getMessage()); } - if(parsedIpAddress.isIPv4() && parsedCidrBlock.isIPv6() || parsedIpAddress.isIPv6() && parsedCidrBlock.isIPv4()) { - throw new RuntimeException("The given ipAddress '"+ipAddress+"' and cidrBlock '"+cidrBlock+"' are not compatible. Both must be either IPv4 or IPv6."); + if (parsedIpAddress.isIPv4() && parsedCidrBlock.isIPv6() || parsedIpAddress.isIPv6() && parsedCidrBlock.isIPv4()) { + throw new RuntimeException("The given ipAddress '" + ipAddress + "' and cidrBlock '" + cidrBlock + "' are not compatible. Both must be either IPv4 or IPv6."); } return parsedCidrBlock.contains(parsedIpAddress); } }; - abstract class SerializableAbstractFunction2 extends AbstractFunction2 - implements Serializable { + /** + * get the function reference according to its name + * + * @param funcName + * @return + */ + static ScalaUDF visit(String funcName, List expressions) { + switch (funcName) { + case "cidr": + return new ScalaUDF(cidrFunction, + DataTypes.BooleanType, + seq(expressions), + seq(), + Option.empty(), + Option.apply("cidr"), + false, + true); + case "json_delete": + return new ScalaUDF(jsonDeleteFunction, + DataTypes.StringType, + seq(expressions), + seq(), + Option.empty(), + Option.apply("json_delete"), + false, + true); + case "json_append": + return new ScalaUDF(jsonAppendFunction, + DataTypes.StringType, + seq(expressions), + seq(), + Option.empty(), + Option.apply("json_append"), + false, + true); + default: + return null; + } } } diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystExpressionVisitor.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystExpressionVisitor.java index bc14ba9d4..d9ace48ba 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystExpressionVisitor.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/CatalystExpressionVisitor.java @@ -11,29 +11,21 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedStar$; import org.apache.spark.sql.catalyst.expressions.CaseWhen; import org.apache.spark.sql.catalyst.expressions.Cast$; -import org.apache.spark.sql.catalyst.expressions.CurrentRow$; import org.apache.spark.sql.catalyst.expressions.Exists$; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.GreaterThanOrEqual; import org.apache.spark.sql.catalyst.expressions.In$; import org.apache.spark.sql.catalyst.expressions.InSubquery$; import org.apache.spark.sql.catalyst.expressions.LambdaFunction$; -import org.apache.spark.sql.catalyst.expressions.LessThan; import org.apache.spark.sql.catalyst.expressions.LessThanOrEqual; import org.apache.spark.sql.catalyst.expressions.ListQuery$; import org.apache.spark.sql.catalyst.expressions.MakeInterval$; import org.apache.spark.sql.catalyst.expressions.NamedExpression; import org.apache.spark.sql.catalyst.expressions.Predicate; -import org.apache.spark.sql.catalyst.expressions.RowFrame$; -import org.apache.spark.sql.catalyst.expressions.ScalaUDF; import org.apache.spark.sql.catalyst.expressions.ScalarSubquery$; import org.apache.spark.sql.catalyst.expressions.UnresolvedNamedLambdaVariable; import org.apache.spark.sql.catalyst.expressions.UnresolvedNamedLambdaVariable$; -import org.apache.spark.sql.catalyst.expressions.SpecifiedWindowFrame; -import org.apache.spark.sql.catalyst.expressions.WindowExpression; -import org.apache.spark.sql.catalyst.expressions.WindowSpecDefinition; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.apache.spark.sql.types.DataTypes; import org.opensearch.sql.ast.AbstractNodeVisitor; import org.opensearch.sql.ast.expression.AggregateFunction; import org.opensearch.sql.ast.expression.Alias; @@ -44,7 +36,6 @@ import org.opensearch.sql.ast.expression.Case; import org.opensearch.sql.ast.expression.Cast; import org.opensearch.sql.ast.expression.Compare; -import org.opensearch.sql.ast.expression.DataType; import org.opensearch.sql.ast.expression.FieldsMapping; import org.opensearch.sql.ast.expression.Function; import org.opensearch.sql.ast.expression.In; @@ -68,9 +59,7 @@ import org.opensearch.sql.ast.tree.FillNull; import org.opensearch.sql.ast.tree.Kmeans; import org.opensearch.sql.ast.tree.RareTopN; -import org.opensearch.sql.ast.tree.Trendline; import org.opensearch.sql.ast.tree.UnresolvedPlan; -import org.opensearch.sql.expression.function.BuiltinFunctionName; import org.opensearch.sql.expression.function.SerializableUdf; import org.opensearch.sql.ppl.utils.AggregatorTransformer; import org.opensearch.sql.ppl.utils.BuiltinFunctionTransformer; @@ -89,6 +78,7 @@ import java.util.stream.Collectors; import static java.util.Collections.emptyList; +import static java.util.List.of; import static org.opensearch.sql.expression.function.BuiltinFunctionName.EQUAL; import static org.opensearch.sql.ppl.CatalystPlanContext.findRelation; import static org.opensearch.sql.ppl.utils.BuiltinFunctionTransformer.createIntervalArgs; @@ -438,17 +428,7 @@ public Expression visitCidr(org.opensearch.sql.ast.expression.Cidr node, Catalys Expression ipAddressExpression = context.getNamedParseExpressions().pop(); analyze(node.getCidrBlock(), context); Expression cidrBlockExpression = context.getNamedParseExpressions().pop(); - - ScalaUDF udf = new ScalaUDF(SerializableUdf.cidrFunction, - DataTypes.BooleanType, - seq(ipAddressExpression,cidrBlockExpression), - seq(), - Option.empty(), - Option.apply("cidr"), - false, - true); - - return context.getNamedParseExpressions().push(udf); + return context.getNamedParseExpressions().push(SerializableUdf.visit("cidr", of(ipAddressExpression,cidrBlockExpression))); } @Override diff --git a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/BuiltinFunctionTransformer.java b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/BuiltinFunctionTransformer.java index 0a4f19b53..f73a1c491 100644 --- a/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/BuiltinFunctionTransformer.java +++ b/ppl-spark-integration/src/main/java/org/opensearch/sql/ppl/utils/BuiltinFunctionTransformer.java @@ -13,12 +13,15 @@ import org.apache.spark.sql.catalyst.expressions.DateAddInterval$; import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.Literal$; +import org.apache.spark.sql.catalyst.expressions.ScalaUDF; import org.apache.spark.sql.catalyst.expressions.TimestampAdd$; import org.apache.spark.sql.catalyst.expressions.TimestampDiff$; import org.apache.spark.sql.catalyst.expressions.ToUTCTimestamp$; import org.apache.spark.sql.catalyst.expressions.UnaryMinus$; import org.opensearch.sql.ast.expression.IntervalUnit; import org.opensearch.sql.expression.function.BuiltinFunctionName; +import org.opensearch.sql.expression.function.SerializableUdf; +import org.opensearch.sql.ppl.CatalystPlanContext; import scala.Option; import java.util.Arrays; @@ -26,7 +29,6 @@ import java.util.Map; import java.util.function.Function; -import static org.opensearch.flint.spark.ppl.OpenSearchPPLLexer.DISTINCT_COUNT_APPROX; import static org.opensearch.sql.expression.function.BuiltinFunctionName.ADD; import static org.opensearch.sql.expression.function.BuiltinFunctionName.ADDDATE; import static org.opensearch.sql.expression.function.BuiltinFunctionName.APPROX_COUNT_DISTINCT; @@ -76,7 +78,7 @@ public interface BuiltinFunctionTransformer { * This is only used for the built-in functions between PPL and Spark with different names. * If the built-in function names are the same in PPL and Spark, add it to {@link BuiltinFunctionName} only. */ - static final Map SPARK_BUILTIN_FUNCTION_NAME_MAPPING + Map SPARK_BUILTIN_FUNCTION_NAME_MAPPING = ImmutableMap.builder() // arithmetic operators .put(ADD, "+") @@ -117,7 +119,7 @@ public interface BuiltinFunctionTransformer { /** * The name mapping between PPL builtin functions to Spark builtin functions. */ - static final Map, Expression>> PPL_TO_SPARK_FUNC_MAPPING + Map, Expression>> PPL_TO_SPARK_FUNC_MAPPING = ImmutableMap., Expression>>builder() // json functions .put( @@ -176,9 +178,11 @@ public interface BuiltinFunctionTransformer { static Expression builtinFunction(org.opensearch.sql.ast.expression.Function function, List args) { if (BuiltinFunctionName.of(function.getFuncName()).isEmpty()) { - // TODO change it when UDF is supported - // TODO should we support more functions which are not PPL builtin functions. E.g Spark builtin functions - throw new UnsupportedOperationException(function.getFuncName() + " is not a builtin function of PPL"); + ScalaUDF udf = SerializableUdf.visit(function.getFuncName(), args); + if(udf == null) { + throw new UnsupportedOperationException(function.getFuncName() + " is not a builtin function of PPL"); + } + return udf; } else { BuiltinFunctionName builtin = BuiltinFunctionName.of(function.getFuncName()).get(); String name = SPARK_BUILTIN_FUNCTION_NAME_MAPPING.get(builtin); diff --git a/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableUdfTest.java b/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableIPUdfTest.java similarity index 87% rename from ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableUdfTest.java rename to ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableIPUdfTest.java index 3d3940730..c11c832c3 100644 --- a/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableUdfTest.java +++ b/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableIPUdfTest.java @@ -1,9 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ package org.opensearch.sql.expression.function; import org.junit.Assert; import org.junit.Test; -public class SerializableUdfTest { +import java.util.Arrays; +import java.util.Collections; + +import static java.util.Collections.singletonList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class SerializableIPUdfTest { @Test(expected = RuntimeException.class) public void cidrNullIpTest() { diff --git a/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableJsonUdfTest.java b/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableJsonUdfTest.java new file mode 100644 index 000000000..fb47803cf --- /dev/null +++ b/ppl-spark-integration/src/test/java/org/opensearch/sql/expression/function/SerializableJsonUdfTest.java @@ -0,0 +1,153 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.sql.expression.function; + +import org.junit.Test; +import scala.collection.mutable.WrappedArray; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.opensearch.sql.expression.function.SerializableUdf.jsonAppendFunction; +import static org.opensearch.sql.expression.function.SerializableUdf.jsonDeleteFunction; + +public class SerializableJsonUdfTest { + + @Test + public void testJsonDeleteFunctionRemoveSingleKey() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":\"value2\",\"key3\":\"value3\"}"; + String expectedJson = "{\"key1\":\"value1\",\"key3\":\"value3\"}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"key2"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionRemoveNestedKey() { + // Correctly escape double quotes within the JSON string + String jsonStr = "{\"key1\":\"value1\",\"key2\":{ \"key3\":\"value3\",\"key4\":\"value4\" }}"; + String expectedJson = "{\"key1\":\"value1\",\"key2\":{\"key4\":\"value4\"}}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"key2.key3"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionRemoveSingleArrayedKey() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":\"value2\",\"keyArray\":[\"value1\",\"value2\"]}"; + String expectedJson = "{\"key1\":\"value1\",\"key2\":\"value2\"}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"keyArray"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionRemoveMultipleKeys() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":\"value2\",\"key3\":\"value3\"}"; + String expectedJson = "{\"key3\":\"value3\"}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"key1", "key2"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionRemoveMultipleSomeAreNestedKeys() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":{ \"key3\":\"value3\",\"key4\":\"value4\" }}"; + String expectedJson = "{\"key2\":{\"key3\":\"value3\"}}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"key1", "key2.key4"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionRemoveMultipleKeysNestedArrayKeys() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":[{ \"a\":\"valueA\",\"key3\":\"value3\"}, {\"a\":\"valueA\",\"key4\":\"value4\"}]}"; + String expectedJson = "{\"key2\":[{\"key3\":\"value3\"},{\"key4\":\"value4\"}]}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[]{"key1", "key2.a"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonDeleteFunctionNoKeysRemoved() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":\"value2\"}"; + String result = jsonDeleteFunction.apply(jsonStr, WrappedArray.make(new String[0])); + assertEquals(jsonStr, result); + } + + @Test + public void testJsonDeleteFunctionNullJson() { + String result = jsonDeleteFunction.apply(null, WrappedArray.make(new String[]{"key1"})); + assertNull(result); + } + + @Test + public void testJsonDeleteFunctionInvalidJson() { + String invalidJson = "invalid_json"; + String result = jsonDeleteFunction.apply(invalidJson, WrappedArray.make(new String[]{"key1"})); + assertNull(result); + } + + @Test + public void testJsonAppendFunctionAppendToExistingArray() { + String jsonStr = "{\"arrayKey\":[\"value1\",\"value2\"]}"; + String expectedJson = "{\"arrayKey\":[\"value1\",\"value2\",\"value3\"]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"arrayKey", "value3"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionAppendObjectToExistingArray() { + String jsonStr = "{\"key1\":\"value1\",\"key2\":[{\"a\":\"valueA\",\"key3\":\"value3\"}]}"; + String expectedJson = "{\"key1\":\"value1\",\"key2\":[{\"a\":\"valueA\",\"key3\":\"value3\"},{\"a\":\"valueA\",\"key4\":\"value4\"}]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"key2", "{\"a\":\"valueA\",\"key4\":\"value4\"}"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionAddNewArray() { + String jsonStr = "{\"key1\":\"value1\",\"newArray\":[]}"; + String expectedJson = "{\"key1\":\"value1\",\"newArray\":[\"newValue\"]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"newArray", "newValue"})); + assertEquals(expectedJson, result); + } + @Test + public void testJsonAppendFunctionNoSuchKey() { + String jsonStr = "{\"key1\":\"value1\"}"; + String expectedJson = "{\"key1\":\"value1\",\"newKey\":[\"newValue\"]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"newKey", "newValue"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionIgnoreNonArrayKey() { + String jsonStr = "{\"key1\":\"value1\"}"; + String expectedJson = jsonStr; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"key1", "newValue"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionWithNestedArrayKeys() { + String jsonStr = "{\"key2\":[{\"a\":[\"Value1\"],\"key3\":\"Value3\"},{\"a\":[\"Value1\"],\"key4\":\"Value4\"}]}"; + String expectedJson = "{\"key2\":[{\"a\":[\"Value1\",\"Value2\"],\"key3\":\"Value3\"},{\"a\":[\"Value1\",\"Value2\"],\"key4\":\"Value4\"}]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"key2.a","Value2"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionWithObjectKey() { + String jsonStr = "{\"key2\":[{\"a\":[\"Value1\"],\"key3\":\"Value3\"},{\"a\":[\"Value1\"],\"key4\":\"Value4\"}]}"; + String expectedJson = "{\"key2\":[{\"a\":[\"Value1\"],\"key3\":\"Value3\"},{\"a\":[\"Value1\"],\"key4\":\"Value4\"},\"Value2\"]}"; + String result = jsonAppendFunction.apply(jsonStr, WrappedArray.make(new String[]{"key2","Value2"})); + assertEquals(expectedJson, result); + } + + @Test + public void testJsonAppendFunctionNullJson() { + String result = jsonAppendFunction.apply(null, WrappedArray.make(new String[]{"key1", "newValue"})); + assertNull(result); + } + + @Test + public void testJsonAppendFunctionInvalidJson() { + String invalidJson = "invalid_json"; + String result = jsonAppendFunction.apply(invalidJson, WrappedArray.make(new String[]{"key1", "newValue"})); + assertNull(result); + } +} diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanJsonFunctionsTranslatorTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanJsonFunctionsTranslatorTestSuite.scala index 6193bc43f..fae070a75 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanJsonFunctionsTranslatorTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanJsonFunctionsTranslatorTestSuite.scala @@ -5,15 +5,21 @@ package org.opensearch.flint.spark.ppl +import java.util + import org.opensearch.flint.spark.ppl.PlaneUtils.plan +import org.opensearch.sql.expression.function.SerializableUdf +import org.opensearch.sql.expression.function.SerializableUdf.visit import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} +import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq import org.scalatest.matchers.should.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} -import org.apache.spark.sql.catalyst.expressions.{EqualTo, Literal} +import org.apache.spark.sql.catalyst.expressions.{Alias, EqualTo, Literal} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{Filter, Project} +import org.apache.spark.sql.types.DataTypes class PPLLogicalPlanJsonFunctionsTranslatorTestSuite extends SparkFunSuite @@ -185,6 +191,51 @@ class PPLLogicalPlanJsonFunctionsTranslatorTestSuite comparePlans(expectedPlan, logPlan, false) } + test("test json_delete()") { + val context = new CatalystPlanContext + val logPlan = + planTransformer.visit( + plan( + pplParser, + """source=t | eval result = json_delete('{"a":[{"b":1},{"c":2}]}', array('a.b'))"""), + context) + + val table = UnresolvedRelation(Seq("t")) + val keysExpression = + UnresolvedFunction("array", Seq(Literal("a.b")), isDistinct = false) + val jsonObjExp = + Literal("""{"a":[{"b":1},{"c":2}]}""") + val jsonFunc = + Alias(visit("json_delete", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val expectedPlan = Project(Seq(UnresolvedStar(None)), eval) + comparePlans(expectedPlan, logPlan, false) + } + + test("test json_append()") { + val context = new CatalystPlanContext + val logPlan = + planTransformer.visit( + plan( + pplParser, + """source=t | eval result = json_append('{"a":[{"b":1},{"c":2}]}', array('a.b','c','d'))"""), + context) + + val table = UnresolvedRelation(Seq("t")) + val keysExpression = + UnresolvedFunction( + "array", + Seq(Literal("a.b"), Literal("c"), Literal("d")), + isDistinct = false) + val jsonObjExp = + Literal("""{"a":[{"b":1},{"c":2}]}""") + val jsonFunc = + Alias(visit("json_append", util.List.of(jsonObjExp, keysExpression)), "result")() + val eval = Project(Seq(UnresolvedStar(None), jsonFunc), table) + val expectedPlan = Project(Seq(UnresolvedStar(None)), eval) + comparePlans(expectedPlan, logPlan, false) + } + test("test json_keys()") { val context = new CatalystPlanContext val logPlan = diff --git a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanParseCidrmatchTestSuite.scala b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanParseCidrmatchTestSuite.scala index 213f201cc..c8a8a67ad 100644 --- a/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanParseCidrmatchTestSuite.scala +++ b/ppl-spark-integration/src/test/scala/org/opensearch/flint/spark/ppl/PPLLogicalPlanParseCidrmatchTestSuite.scala @@ -7,13 +7,14 @@ package org.opensearch.flint.spark.ppl import org.opensearch.flint.spark.ppl.PlaneUtils.plan import org.opensearch.sql.expression.function.SerializableUdf +import org.opensearch.sql.expression.function.SerializableUdf.visit import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq import org.scalatest.matchers.should.Matchers import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} -import org.apache.spark.sql.catalyst.expressions.{Alias, And, Ascending, CaseWhen, Descending, EqualTo, GreaterThan, Literal, NullsFirst, NullsLast, RegExpExtract, ScalaUDF, SortOrder} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Ascending, CaseWhen, Descending, EqualTo, GreaterThan, Literal, NullsFirst, NullsLast, RegExpExtract, SortOrder} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.DataTypes @@ -41,15 +42,7 @@ class PPLLogicalPlanParseCidrmatchTestSuite val filterIpv6 = EqualTo(UnresolvedAttribute("isV6"), Literal(false)) val filterIsValid = EqualTo(UnresolvedAttribute("isValid"), Literal(true)) - val cidr = ScalaUDF( - SerializableUdf.cidrFunction, - DataTypes.BooleanType, - seq(ipAddress, cidrExpression), - seq(), - Option.empty, - Option.apply("cidr"), - false, - true) + val cidr = visit("cidr", java.util.List.of(ipAddress, cidrExpression)) val expectedPlan = Project( Seq(UnresolvedStar(None)), @@ -71,15 +64,7 @@ class PPLLogicalPlanParseCidrmatchTestSuite val filterIpv6 = EqualTo(UnresolvedAttribute("isV6"), Literal(true)) val filterIsValid = EqualTo(UnresolvedAttribute("isValid"), Literal(false)) - val cidr = ScalaUDF( - SerializableUdf.cidrFunction, - DataTypes.BooleanType, - seq(ipAddress, cidrExpression), - seq(), - Option.empty, - Option.apply("cidr"), - false, - true) + val cidr = visit("cidr", java.util.List.of(ipAddress, cidrExpression)) val expectedPlan = Project( Seq(UnresolvedStar(None)), @@ -100,15 +85,7 @@ class PPLLogicalPlanParseCidrmatchTestSuite val cidrExpression = Literal("2003:db8::/32") val filterIpv6 = EqualTo(UnresolvedAttribute("isV6"), Literal(true)) - val cidr = ScalaUDF( - SerializableUdf.cidrFunction, - DataTypes.BooleanType, - seq(ipAddress, cidrExpression), - seq(), - Option.empty, - Option.apply("cidr"), - false, - true) + val cidr = visit("cidr", java.util.List.of(ipAddress, cidrExpression)) val expectedPlan = Project( Seq(UnresolvedAttribute("ip")), @@ -130,15 +107,7 @@ class PPLLogicalPlanParseCidrmatchTestSuite val filterIpv6 = EqualTo(UnresolvedAttribute("isV6"), Literal(true)) val filterClause = Filter(filterIpv6, UnresolvedRelation(Seq("t"))) - val cidr = ScalaUDF( - SerializableUdf.cidrFunction, - DataTypes.BooleanType, - seq(ipAddress, cidrExpression), - seq(), - Option.empty, - Option.apply("cidr"), - false, - true) + val cidr = visit("cidr", java.util.List.of(ipAddress, cidrExpression)) val equalTo = EqualTo(Literal(true), cidr) val caseFunction = CaseWhen(Seq((equalTo, Literal("in"))), Literal("out")) From 957de4e27d3a243ccb9beccd2ee72ab231e9dcf4 Mon Sep 17 00:00:00 2001 From: normanj-bitquill <78755797+normanj-bitquill@users.noreply.github.com> Date: Tue, 17 Dec 2024 13:18:27 -0800 Subject: [PATCH 3/3] Created docker files for an integ test cluster (#601) (#986) * Created docker files for an integ test cluster (#601) Cluster contains: * Spark master * Spark worker * OpenSearch server * OpenSearch dashboards * Minio server Signed-off-by: Norman Jordan * Updated to start Spark Connect on the Spark master container Signed-off-by: Norman Jordan * Can integration tests against the docker cluster The Python script for integration tests was updated to run queries against the docker cluster. The required indices are created as part of the script. The queries for the Python script were likely out of date. These have been updated when the fix for the query was obvious. There are still 6 tests that fail. Signed-off-by: Norman Jordan * Fixed up the documentation for docker integration tests Signed-off-by: Norman Jordan * Added a link in the toplevel README Signed-off-by: Norman Jordan * Described creation of test indices Signed-off-by: Norman Jordan --------- Signed-off-by: Norman Jordan --- README.md | 2 + docker/integ-test/.env | 13 + docker/integ-test/docker-compose.yml | 143 ++++ docker/integ-test/log4j2.properties | 69 ++ docker/integ-test/prepare_scala_queries.py | 23 + docker/integ-test/queries.scala | 619 +++++++++++++++++ docker/integ-test/spark-defaults.conf | 35 + docker/integ-test/spark-master-entrypoint.sh | 17 + integ-test/script/README.md | 73 +- integ-test/script/SanityTest.py | 202 ++---- integ-test/script/data/customer.mapping.json | 30 + integ-test/script/data/http_logs.json | 12 + integ-test/script/data/http_logs.mapping.json | 30 + integ-test/script/data/lineitem.mapping.json | 54 ++ integ-test/script/data/nation.mapping.json | 18 + integ-test/script/data/nested.json | 10 + integ-test/script/data/nested.mapping.json | 37 + integ-test/script/data/orders.mapping.json | 33 + integ-test/script/data/part.mapping.json | 33 + integ-test/script/data/partsupp.mapping.json | 21 + integ-test/script/data/people.json | 12 + integ-test/script/data/people.mapping.json | 24 + integ-test/script/data/region.mapping.json | 15 + integ-test/script/data/supplier.mapping.json | 27 + integ-test/script/data/work_info.json | 10 + integ-test/script/data/work_info.mapping.json | 18 + integ-test/script/test_cases.csv | 644 +++++++++--------- 27 files changed, 1748 insertions(+), 476 deletions(-) create mode 100644 docker/integ-test/.env create mode 100644 docker/integ-test/docker-compose.yml create mode 100644 docker/integ-test/log4j2.properties create mode 100755 docker/integ-test/prepare_scala_queries.py create mode 100644 docker/integ-test/queries.scala create mode 100644 docker/integ-test/spark-defaults.conf create mode 100755 docker/integ-test/spark-master-entrypoint.sh create mode 100644 integ-test/script/data/customer.mapping.json create mode 100644 integ-test/script/data/http_logs.json create mode 100644 integ-test/script/data/http_logs.mapping.json create mode 100644 integ-test/script/data/lineitem.mapping.json create mode 100644 integ-test/script/data/nation.mapping.json create mode 100644 integ-test/script/data/nested.json create mode 100644 integ-test/script/data/nested.mapping.json create mode 100644 integ-test/script/data/orders.mapping.json create mode 100644 integ-test/script/data/part.mapping.json create mode 100644 integ-test/script/data/partsupp.mapping.json create mode 100644 integ-test/script/data/people.json create mode 100644 integ-test/script/data/people.mapping.json create mode 100644 integ-test/script/data/region.mapping.json create mode 100644 integ-test/script/data/supplier.mapping.json create mode 100644 integ-test/script/data/work_info.json create mode 100644 integ-test/script/data/work_info.mapping.json diff --git a/README.md b/README.md index db3790e64..6732db3af 100644 --- a/README.md +++ b/README.md @@ -90,6 +90,8 @@ bin/spark-shell --packages "org.opensearch:opensearch-spark-ppl_2.12:0.7.0-SNAPS ### PPL Run queries on a local spark cluster See ppl usage sample on local spark cluster [PPL on local spark ](docs/ppl-lang/local-spark-ppl-test-instruction.md) +### Running integration tests on a local spark cluster +See integration test documentation [Docker Integration Tests](integ-test/script/README.md) ## Code of Conduct diff --git a/docker/integ-test/.env b/docker/integ-test/.env new file mode 100644 index 000000000..cf73bdc89 --- /dev/null +++ b/docker/integ-test/.env @@ -0,0 +1,13 @@ +SPARK_VERSION=3.5.3 +OPENSEARCH_VERSION=latest +DASHBOARDS_VERSION=latest +MASTER_UI_PORT=8080 +MASTER_PORT=7077 +UI_PORT=4040 +SPARK_CONNECT_PORT=15002 +PPL_JAR=../../ppl-spark-integration/target/scala-2.12/ppl-spark-integration-assembly-0.7.0-SNAPSHOT.jar +FLINT_JAR=../../flint-spark-integration/target/scala-2.12/flint-spark-integration-assembly-0.7.0-SNAPSHOT.jar +OPENSEARCH_NODE_MEMORY=512m +OPENSEARCH_ADMIN_PASSWORD=C0rrecthorsebatterystaple. +OPENSEARCH_PORT=9200 +OPENSEARCH_DASHBOARDS_PORT=5601 diff --git a/docker/integ-test/docker-compose.yml b/docker/integ-test/docker-compose.yml new file mode 100644 index 000000000..c5ee53d7d --- /dev/null +++ b/docker/integ-test/docker-compose.yml @@ -0,0 +1,143 @@ +services: + spark: + image: bitnami/spark:${SPARK_VERSION:-3.5.3} + container_name: spark + ports: + - "${MASTER_UI_PORT:-8080}:8080" + - "${MASTER_PORT:-7077}:7077" + - "${UI_PORT:-4040}:4040" + - "${SPARK_CONNECT_PORT}:15002" + entrypoint: /opt/bitnami/scripts/spark/master-entrypoint.sh + environment: + - SPARK_MODE=master + - SPARK_RPC_AUTHENTICATION_ENABLED=no + - SPARK_RPC_ENCRYPTION_ENABLED=no + - SPARK_LOCAL_STORAGE_ENCRYPTION_ENABLED=no + - SPARK_SSL_ENABLED=no + - SPARK_PUBLIC_DNS=localhost + volumes: + - type: bind + source: ./spark-master-entrypoint.sh + target: /opt/bitnami/scripts/spark/master-entrypoint.sh + - type: bind + source: ./spark-defaults.conf + target: /opt/bitnami/spark/conf/spark-defaults.conf + - type: bind + source: ./log4j2.properties + target: /opt/bitnami/spark/conf/log4j2.properties + - type: bind + source: $PPL_JAR + target: /opt/bitnami/spark/jars/ppl-spark-integration.jar + - type: bind + source: $FLINT_JAR + target: /opt/bitnami/spark/jars/flint-spark-integration.jar + healthcheck: + test: ["CMD", "curl", "-f", "http://localhost:8080/"] + interval: 1m + timeout: 5s + retries: 3 + start_period: 30s + start_interval: 5s + networks: + - opensearch-net + + spark-worker: + image: bitnami/spark:${SPARK_VERSION:-3.5.3} + container_name: spark-worker + environment: + - SPARK_MODE=worker + - SPARK_MASTER_URL=spark://spark:7077 + - SPARK_WORKER_MEMORY=${WORKER_MEMORY:-1G} + - SPARK_WORKER_CORES=${WORKER_CORES:-1} + - SPARK_RPC_AUTHENTICATION_ENABLED=no + - SPARK_RPC_ENCRYPTION_ENABLED=no + - SPARK_LOCAL_STORAGE_ENCRYPTION_ENABLED=no + - SPARK_SSL_ENABLED=no + - SPARK_PUBLIC_DNS=localhost + volumes: + - type: bind + source: ./spark-defaults.conf + target: /opt/bitnami/spark/conf/spark-defaults.conf + - type: bind + source: ./log4j2.properties + target: /opt/bitnami/spark/conf/log4j2.properties + - type: bind + source: $PPL_JAR + target: /opt/bitnami/spark/jars/ppl-spark-integration.jar + - type: bind + source: $FLINT_JAR + target: /opt/bitnami/spark/jars/flint-spark-integration.jar + networks: + - opensearch-net + depends_on: + - spark + + opensearch: + image: opensearchproject/opensearch:${OPENSEARCH_VERSION:-latest} + container_name: opensearch + environment: + - cluster.name=opensearch-cluster + - node.name=opensearch + - discovery.seed_hosts=opensearch + - cluster.initial_cluster_manager_nodes=opensearch + - bootstrap.memory_lock=true + - plugins.security.ssl.http.enabled=false + - OPENSEARCH_JAVA_OPTS=-Xms${OPENSEARCH_NODE_MEMORY:-512m} -Xmx${OPENSEARCH_NODE_MEMORY:-512m} + - OPENSEARCH_INITIAL_ADMIN_PASSWORD=${OPENSEARCH_ADMIN_PASSWORD} + ulimits: + memlock: + soft: -1 + hard: -1 + nofile: + soft: 65536 + hard: 65536 + volumes: + - opensearch-data:/usr/share/opensearch/data + ports: + - ${OPENSEARCH_PORT:-9200}:9200 + - 9600:9600 + expose: + - "${OPENSEARCH_PORT:-9200}" + healthcheck: + test: ["CMD", "curl", "-f", "-u", "admin:${OPENSEARCH_ADMIN_PASSWORD}", "http://localhost:9200/_cluster/health"] + interval: 1m + timeout: 5s + retries: 3 + start_period: 30s + start_interval: 5s + networks: + - opensearch-net + + opensearch-dashboards: + image: opensearchproject/opensearch-dashboards:${DASHBOARDS_VERSION} + container_name: opensearch-dashboards + ports: + - ${OPENSEARCH_DASHBOARDS_PORT:-5601}:5601 + expose: + - "${OPENSEARCH_DASHBOARDS_PORT:-5601}" + environment: + OPENSEARCH_HOSTS: '["http://opensearch:9200"]' + networks: + - opensearch-net + depends_on: + - opensearch + + minio: + image: minio/minio + container_name: minio-S3 + # See original entrypoint/command under https://github.com/minio/minio/blob/master/Dockerfile + entrypoint: sh -c 'mkdir -p /data/test && minio server /data --console-address ":9001"' + ports: + - "9000:9000" + - "9001:9001" + volumes: + - minio-data:/data + networks: + - opensearch-net + +volumes: + opensearch-data: + minio-data: + +networks: + opensearch-net: diff --git a/docker/integ-test/log4j2.properties b/docker/integ-test/log4j2.properties new file mode 100644 index 000000000..ab96e03ba --- /dev/null +++ b/docker/integ-test/log4j2.properties @@ -0,0 +1,69 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the console +rootLogger.level = info +rootLogger.appenderRef.stdout.ref = console + +# In the pattern layout configuration below, we specify an explicit `%ex` conversion +# pattern for logging Throwables. If this was omitted, then (by default) Log4J would +# implicitly add an `%xEx` conversion pattern which logs stacktraces with additional +# class packaging information. That extra information can sometimes add a substantial +# performance overhead, so we disable it in our default logging config. +# For more information, see SPARK-39361. +appender.console.type = Console +appender.console.name = console +appender.console.target = SYSTEM_ERR +appender.console.layout.type = PatternLayout +appender.console.layout.pattern = %d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n%ex + +# Set the default spark-shell/spark-sql log level to WARN. When running the +# spark-shell/spark-sql, the log level for these classes is used to overwrite +# the root logger's log level, so that the user can have different defaults +# for the shell and regular Spark apps. +logger.repl.name = org.apache.spark.repl.Main +logger.repl.level = warn + +logger.thriftserver.name = org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver +logger.thriftserver.level = warn + +# Settings to quiet third party logs that are too verbose +logger.jetty1.name = org.sparkproject.jetty +logger.jetty1.level = warn +logger.jetty2.name = org.sparkproject.jetty.util.component.AbstractLifeCycle +logger.jetty2.level = error +logger.replexprTyper.name = org.apache.spark.repl.SparkIMain$exprTyper +logger.replexprTyper.level = info +logger.replSparkILoopInterpreter.name = org.apache.spark.repl.SparkILoop$SparkILoopInterpreter +logger.replSparkILoopInterpreter.level = info +logger.parquet1.name = org.apache.parquet +logger.parquet1.level = error +logger.parquet2.name = parquet +logger.parquet2.level = error + +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +logger.RetryingHMSHandler.name = org.apache.hadoop.hive.metastore.RetryingHMSHandler +logger.RetryingHMSHandler.level = fatal +logger.FunctionRegistry.name = org.apache.hadoop.hive.ql.exec.FunctionRegistry +logger.FunctionRegistry.level = error + +# For deploying Spark ThriftServer +# SPARK-34128: Suppress undesirable TTransportException warnings involved in THRIFT-4805 +appender.console.filter.1.type = RegexFilter +appender.console.filter.1.regex = .*Thrift error occurred during processing of message.* +appender.console.filter.1.onMatch = deny +appender.console.filter.1.onMismatch = neutral diff --git a/docker/integ-test/prepare_scala_queries.py b/docker/integ-test/prepare_scala_queries.py new file mode 100755 index 000000000..dec62593b --- /dev/null +++ b/docker/integ-test/prepare_scala_queries.py @@ -0,0 +1,23 @@ +#!/usr/bin/env python3 + +import csv + +queries = None +with open('../../integ-test/script/test_cases.csv', 'r') as f: + reader = csv.DictReader(f) + queries = [(row['query'], i, row.get('expected_status', None)) for i, row in enumerate(reader, start=1) if row['query'].strip()] + +print('try {') +for query in queries: + query_str = query[0].replace('\n', '').replace('"', '\\"') + if 'FAILED' == query[2]: + print(' try {') + print(f' spark.sql("{query_str}")') + print(' throw new Error') + print(' } catch {') + print(' case e: Exception => null') + print(' }\n') + else: + print(f' spark.sql("{query_str}")\n') +print('}') + diff --git a/docker/integ-test/queries.scala b/docker/integ-test/queries.scala new file mode 100644 index 000000000..7d6ee78c1 --- /dev/null +++ b/docker/integ-test/queries.scala @@ -0,0 +1,619 @@ +{ + try { + spark.sql("describe myglue_test.default.http_logs") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("describe `myglue_test`.`default`.`http_logs`") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | dedup 1 status | fields @timestamp, clientip, status, size | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup status, size | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup 1 status keepempty=true | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup status, size keepempty=true | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup 2 status | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup 2 status, size | head 10") + + spark.sql("source = myglue_test.default.http_logs | dedup 2 status, size keepempty=true | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | dedup status CONSECUTIVE=true | fields status") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | dedup 2 status, size CONSECUTIVE=true | fields status") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | sort stat | fields @timestamp, clientip, status | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | fields @timestamp, notexisted | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.nested | fields int_col, struct_col.field1, struct_col2.field1 | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.nested | where struct_col2.field1.subfield > 'valueA' | sort int_col | fields int_col, struct_col.field1.subfield, struct_col2.field1.subfield") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | fields - @timestamp, clientip, status | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval new_time = @timestamp, new_clientip = clientip | fields - new_time, new_clientip, status | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval new_clientip = lower(clientip) | fields - new_clientip | head 10") + + spark.sql("source = myglue_test.default.http_logs | fields + @timestamp, clientip, status | fields - clientip, status | head 10") + + spark.sql("source = myglue_test.default.http_logs | fields - clientip, status | fields + @timestamp, clientip, status| head 10") + + spark.sql("source = myglue_test.default.http_logs | where status = 200 | head 10") + + spark.sql("source = myglue_test.default.http_logs | where status != 200 | head 10") + + spark.sql("source = myglue_test.default.http_logs | where size > 0 | head 10") + + spark.sql("source = myglue_test.default.http_logs | where size <= 0 | head 10") + + spark.sql("source = myglue_test.default.http_logs | where clientip = '236.14.2.0' | head 10") + + spark.sql("source = myglue_test.default.http_logs | where size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100") + + spark.sql("source = myglue_test.default.http_logs | where size <= 0 AND like(request, 'GET%') | head 10") + + spark.sql("source = myglue_test.default.http_logs status = 200 | head 10") + + spark.sql("source = myglue_test.default.http_logs size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100") + + spark.sql("source = myglue_test.default.http_logs size <= 0 AND like(request, 'GET%') | head 10") + + spark.sql("source = myglue_test.default.http_logs substring(clientip, 5, 2) = \"12\" | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | where isempty(size)") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | where ispresent(size)") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | where isnull(size) | head 10") + + spark.sql("source = myglue_test.default.http_logs | where isnotnull(size) | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | where isnotnull(coalesce(size, status)) | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | where like(request, 'GET%') | head 10") + + spark.sql("source = myglue_test.default.http_logs | where like(request, '%bordeaux%') | head 10") + + spark.sql("source = myglue_test.default.http_logs | where substring(clientip, 5, 2) = \"12\" | head 10") + + spark.sql("source = myglue_test.default.http_logs | where lower(request) = \"get /images/backnews.gif http/1.0\" | head 10") + + spark.sql("source = myglue_test.default.http_logs | where length(request) = 38 | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | where case(status = 200, 'success' else 'failed') = 'success' | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | eval h = \"Hello\", w = \"World\" | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval @h = \"Hello\" | eval @w = \"World\" | fields @timestamp, @h, @w") + + spark.sql("source = myglue_test.default.http_logs | eval newF = clientip | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval newF = clientip | fields clientip, newF | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval f = size | where f > 1 | sort f | fields size, clientip, status | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval f = status * 2 | eval h = f * 2 | fields status, f, h | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval f = size * 2, h = status | stats sum(f) by h") + + spark.sql("source = myglue_test.default.http_logs | eval f = UPPER(request) | eval h = 40 | fields f, h | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | eval request = \"test\" | fields request | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | eval size = abs(size) | where size < 500") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | eval status_string = case(status = 200, 'success' else 'failed') | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | eval n = now() | eval t = unix_timestamp(@timestamp) | fields n, t | head 10") + + try { + spark.sql("source = myglue_test.default.http_logs | eval e = isempty(size) | eval p = ispresent(size) | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | eval c = coalesce(size, status) | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | eval c = coalesce(request) | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | eval col1 = ln(size) | eval col2 = unix_timestamp(@timestamp) | sort - col1 | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval col1 = 1 | sort col1 | head 4 | eval col2 = 2 | sort - col2 | sort - size | head 2 | fields @timestamp, clientip, col2") + + spark.sql("source = myglue_test.default.mini_http_logs | eval stat = status | where stat > 300 | sort stat | fields @timestamp,clientip,status | head 5") + + spark.sql("source = myglue_test.default.http_logs | eval col1 = size, col2 = clientip | stats avg(col1) by col2") + + spark.sql("source = myglue_test.default.http_logs | stats avg(size) by clientip") + + spark.sql("source = myglue_test.default.http_logs | eval new_request = upper(request) | eval compound_field = concat('Hello ', if(like(new_request, '%bordeaux%'), 'World', clientip)) | fields new_request, compound_field | head 10") + + spark.sql("source = myglue_test.default.http_logs | stats avg(size)") + + spark.sql("source = myglue_test.default.nested | stats max(int_col) by struct_col.field2") + + spark.sql("source = myglue_test.default.nested | stats distinct_count(int_col)") + + spark.sql("source = myglue_test.default.nested | stats stddev_samp(int_col)") + + spark.sql("source = myglue_test.default.nested | stats stddev_pop(int_col)") + + spark.sql("source = myglue_test.default.nested | stats percentile(int_col)") + + spark.sql("source = myglue_test.default.nested | stats percentile_approx(int_col)") + + spark.sql("source = myglue_test.default.mini_http_logs | stats stddev_samp(status)") + + spark.sql("source = myglue_test.default.mini_http_logs | where stats > 200 | stats percentile_approx(status, 99)") + + spark.sql("source = myglue_test.default.nested | stats count(int_col) by span(struct_col.field2, 10) as a_span") + + spark.sql("source = myglue_test.default.nested | stats avg(int_col) by span(struct_col.field2, 10) as a_span, struct_col2.field2") + + spark.sql("source = myglue_test.default.http_logs | stats sum(size) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10") + + spark.sql("source = myglue_test.default.http_logs | stats distinct_count(clientip) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10") + + spark.sql("source = myglue_test.default.http_logs | stats avg(size) as avg_size by status, year | stats avg(avg_size) as avg_avg_size by year") + + spark.sql("source = myglue_test.default.http_logs | stats avg(size) as avg_size by status, year, month | stats avg(avg_size) as avg_avg_size by year, month | stats avg(avg_avg_size) as avg_avg_avg_size by year") + + try { + spark.sql("source = myglue_test.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | stats avg(avg_int) as avg_avg_int by struct_col2.field2") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | eval new_col = avg_int | stats avg(avg_int) as avg_avg_int by new_col") + + spark.sql("source = myglue_test.default.nested | rare int_col") + + spark.sql("source = myglue_test.default.nested | rare int_col by struct_col.field2") + + spark.sql("source = myglue_test.default.http_logs | rare request") + + spark.sql("source = myglue_test.default.http_logs | where status > 300 | rare request by status") + + spark.sql("source = myglue_test.default.http_logs | rare clientip") + + spark.sql("source = myglue_test.default.http_logs | where status > 300 | rare clientip") + + spark.sql("source = myglue_test.default.http_logs | where status > 300 | rare clientip by day") + + spark.sql("source = myglue_test.default.nested | top int_col by struct_col.field2") + + spark.sql("source = myglue_test.default.nested | top 1 int_col by struct_col.field2") + + spark.sql("source = myglue_test.default.nested | top 2 int_col by struct_col.field2") + + spark.sql("source = myglue_test.default.nested | top int_col") + + try { + spark.sql("source = myglue_test.default.http_logs | inner join left=l right=r on l.status = r.int_col myglue_test.default.nested | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | fields request, domain | head 10") + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | top 1 domain") + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | stats count() by domain") + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | eval a = 1 | fields a, domain | head 10") + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | where size > 0 | sort - size | fields size, domain | head 10") + + spark.sql("source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/(?[a-zA-Z]+)/.*' | where domain = 'english' | sort - picName | fields domain, picName | head 10") + + spark.sql("source = myglue_test.default.http_logs | patterns request | fields patterns_field | head 10") + + spark.sql("source = myglue_test.default.http_logs | patterns request | where size > 0 | fields patterns_field | head 10") + + spark.sql("source = myglue_test.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | fields request, no_letter | head 10") + + spark.sql("source = myglue_test.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | stats count() by no_letter") + + try { + spark.sql("source = myglue_test.default.http_logs | patterns new_field='status' pattern='[a-zA-Z]' request | fields request, status | head 10") + throw new Error + } catch { + case e: Exception => null + } + + try { + spark.sql("source = myglue_test.default.http_logs | rename @timestamp as timestamp | head 10") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.default.http_logs | sort size | head 10") + + spark.sql("source = myglue_test.default.http_logs | sort + size | head 10") + + spark.sql("source = myglue_test.default.http_logs | sort - size | head 10") + + spark.sql("source = myglue_test.default.http_logs | sort + size, + @timestamp | head 10") + + spark.sql("source = myglue_test.default.http_logs | sort - size, - @timestamp | head 10") + + spark.sql("source = myglue_test.default.http_logs | sort - size, @timestamp | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval c1 = upper(request) | eval c2 = concat('Hello ', if(like(c1, '%bordeaux%'), 'World', clientip)) | eval c3 = length(request) | eval c4 = ltrim(request) | eval c5 = rtrim(request) | eval c6 = substring(clientip, 5, 2) | eval c7 = trim(request) | eval c8 = upper(request) | eval c9 = position('bordeaux' IN request) | eval c10 = replace(request, 'GET', 'GGG') | fields c1, c2, c3, c4, c5, c6, c7, c8, c9, c10 | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval c1 = unix_timestamp(@timestamp) | eval c2 = now() | eval c3 =DAY_OF_WEEK(@timestamp) | eval c4 =DAY_OF_MONTH(@timestamp) | eval c5 =DAY_OF_YEAR(@timestamp) | eval c6 =WEEK_OF_YEAR(@timestamp) | eval c7 =WEEK(@timestamp) | eval c8 =MONTH_OF_YEAR(@timestamp) | eval c9 =HOUR_OF_DAY(@timestamp) | eval c10 =MINUTE_OF_HOUR(@timestamp) | eval c11 =SECOND_OF_MINUTE(@timestamp) | eval c12 =LOCALTIME() | fields c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12 | head 10") + + spark.sql("source=myglue_test.default.people | eval c1 = adddate(@timestamp, 1) | fields c1 | head 10") + + spark.sql("source=myglue_test.default.people | eval c2 = subdate(@timestamp, 1) | fields c2 | head 10") + + spark.sql("source=myglue_test.default.people | eval c1 = date_add(@timestamp INTERVAL 1 DAY) | fields c1 | head 10") + + spark.sql("source=myglue_test.default.people | eval c1 = date_sub(@timestamp INTERVAL 1 DAY) | fields c1 | head 10") + + spark.sql("source=myglue_test.default.people | eval `CURDATE()` = CURDATE() | fields `CURDATE()`") + + spark.sql("source=myglue_test.default.people | eval `CURRENT_DATE()` = CURRENT_DATE() | fields `CURRENT_DATE()`") + + spark.sql("source=myglue_test.default.people | eval `CURRENT_TIMESTAMP()` = CURRENT_TIMESTAMP() | fields `CURRENT_TIMESTAMP()`") + + spark.sql("source=myglue_test.default.people | eval `DATE('2020-08-26')` = DATE('2020-08-26') | fields `DATE('2020-08-26')`") + + spark.sql("source=myglue_test.default.people | eval `DATE(TIMESTAMP('2020-08-26 13:49:00'))` = DATE(TIMESTAMP('2020-08-26 13:49:00')) | fields `DATE(TIMESTAMP('2020-08-26 13:49:00'))`") + + spark.sql("source=myglue_test.default.people | eval `DATE('2020-08-26 13:49')` = DATE('2020-08-26 13:49') | fields `DATE('2020-08-26 13:49')`") + + spark.sql("source=myglue_test.default.people | eval `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')` = DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS'), `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')` = DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a') | fields `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')`, `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')`") + + spark.sql("source=myglue_test.default.people | eval `'2000-01-02' - '2000-01-01'` = DATEDIFF(TIMESTAMP('2000-01-02 00:00:00'), TIMESTAMP('2000-01-01 23:59:59')), `'2001-02-01' - '2004-01-01'` = DATEDIFF(DATE('2001-02-01'), TIMESTAMP('2004-01-01 00:00:00')) | fields `'2000-01-02' - '2000-01-01'`, `'2001-02-01' - '2004-01-01'`") + + spark.sql("source=myglue_test.default.people | eval `DAY(DATE('2020-08-26'))` = DAY(DATE('2020-08-26')) | fields `DAY(DATE('2020-08-26'))`") + + try { + spark.sql("source=myglue_test.default.people | eval `DAYNAME(DATE('2020-08-26'))` = DAYNAME(DATE('2020-08-26')) | fields `DAYNAME(DATE('2020-08-26'))`") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source=myglue_test.default.people | eval `CURRENT_TIMEZONE()` = CURRENT_TIMEZONE() | fields `CURRENT_TIMEZONE()`") + + spark.sql("source=myglue_test.default.people | eval `UTC_TIMESTAMP()` = UTC_TIMESTAMP() | fields `UTC_TIMESTAMP()`") + + spark.sql("source=myglue_test.default.people | eval `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')` = TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00') | eval `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))` = TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00')) | fields `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')`, `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))`") + + spark.sql("source=myglue_test.default.people | eval `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')` = TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00') | eval `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')` = TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00') | fields `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')`, `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')`") + + spark.sql(" source = myglue_test.default.http_logs | stats count()") + + spark.sql("source = myglue_test.default.http_logs | stats avg(size) as c1, max(size) as c2, min(size) as c3, sum(size) as c4, percentile(size, 50) as c5, stddev_pop(size) as c6, stddev_samp(size) as c7, distinct_count(size) as c8") + + spark.sql("source = myglue_test.default.http_logs | eval c1 = abs(size) | eval c2 = ceil(size) | eval c3 = floor(size) | eval c4 = sqrt(size) | eval c5 = ln(size) | eval c6 = pow(size, 2) | eval c7 = mod(size, 2) | fields c1, c2, c3, c4, c5, c6, c7 | head 10") + + spark.sql("source = myglue_test.default.http_logs | eval c1 = isnull(request) | eval c2 = isnotnull(request) | eval c3 = ifnull(request,\"Unknown\") | eval c4 = nullif(request,\"Unknown\") | eval c5 = isnull(size) | eval c6 = if(like(request, '%bordeaux%'), 'hello', 'world') | fields c1, c2, c3, c4, c5, c6 | head 10") + + spark.sql("/* this is block comment */ source = myglue_test.tpch_csv.orders | head 1 // this is line comment") + + spark.sql("/* test in tpch q16, q18, q20 */ source = myglue_test.tpch_csv.orders | head 1 // add source=xx to avoid failure in automation") + + spark.sql("/* test in tpch q4, q21, q22 */ source = myglue_test.tpch_csv.orders | head 1") + + spark.sql("/* test in tpch q2, q11, q15, q17, q20, q22 */ source = myglue_test.tpch_csv.orders | head 1") + + spark.sql("/* test in tpch q7, q8, q9, q13, q15, q22 */ source = myglue_test.tpch_csv.orders | head 1") + + spark.sql("/* lots of inner join tests in tpch */ source = myglue_test.tpch_csv.orders | head 1") + + spark.sql("/* left join test in tpch q13 */ source = myglue_test.tpch_csv.orders | head 1") + + spark.sql("source = myglue_test.tpch_csv.orders | right outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') myglue_test.tpch_csv.customer| stats count(o_orderkey) as c_count by c_custkey| sort - c_count") + + spark.sql("source = myglue_test.tpch_csv.orders | full outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') myglue_test.tpch_csv.customer| stats count(o_orderkey) as c_count by c_custkey| sort - c_count") + + spark.sql("source = myglue_test.tpch_csv.customer| semi join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| where c_mktsegment = 'BUILDING' | sort - c_custkey| head 10") + + spark.sql("source = myglue_test.tpch_csv.customer| anti join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| where c_mktsegment = 'BUILDING' | sort - c_custkey| head 10") + + spark.sql("source = myglue_test.tpch_csv.supplier| where like(s_comment, '%Customer%Complaints%')| join ON s_nationkey > n_nationkey [ source = myglue_test.tpch_csv.nation | where n_name = 'SAUDI ARABIA' ]| sort - s_name| head 10") + + spark.sql("source = myglue_test.tpch_csv.supplier| where like(s_comment, '%Customer%Complaints%')| join [ source = myglue_test.tpch_csv.nation | where n_name = 'SAUDI ARABIA' ]| sort - s_name| head 10") + + spark.sql("source=myglue_test.default.people | LOOKUP myglue_test.default.work_info uid AS id REPLACE department | stats distinct_count(department)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id APPEND department | stats distinct_count(department)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id REPLACE department AS country | stats distinct_count(country)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id APPEND department AS country | stats distinct_count(country)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uID AS id, name REPLACE department | stats distinct_count(department)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS ID, name APPEND department | stats distinct_count(department)") + + spark.sql("source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uID AS id, name | head 10") + + spark.sql("source = myglue_test.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP myglue_test.default.work_info name REPLACE occupation AS major | stats distinct_count(major)") + + spark.sql("source = myglue_test.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP myglue_test.default.work_info name APPEND occupation AS major | stats distinct_count(major)") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('{\"f1\":\"abc\",\"f2\":{\"f3\":\"a\",\"f4\":\"b\"}}') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('[]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json(‘{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('{\"invalid\": \"json\"') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('[1,2,3]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json(‘[1,2') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('[invalid json]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json('invalid json') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json(null) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array('this', 'is', 'a', 'string', 'array') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array() | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array(1, 2, 0, -1, 1.1, -0.11) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array('this', 'is', 1.1, -0.11, true, false) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = array_length(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = array_length(json_array()) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array_length('[]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array_length('[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array_length('{\"key\": 1}') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_array_length('[1,2') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', 'string_value')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', 123.45)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', true)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object(\"a\", 1, \"b\", 2, \"c\", 3)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', array())) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', array(1, 2, 3))) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object('outer', json_object('inner', 123.45))) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = to_json_string(json_object(\"array\", json_array(1,2,0,-1,1.1,-0.11))) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | where json_valid(('{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}') | head 1") + + spark.sql("source = myglue_test.default.http_logs | where not json_valid(('{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}') | head 1") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('{\"account_number\":1,\"balance\":39225,\"age\":32,\"gender\":\"M\"}')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('{\"f1\":\"abc\",\"f2\":{\"f3\":\"a\",\"f4\":\"b\"}}')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2,3,{\"f1\":1,\"f2\":[5,6]},4]')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('[]')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json(‘{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('{\"invalid\": \"json\"')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2,3]')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('[invalid json]')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json('invalid json')) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_keys(json(null)) | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.teacher') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[*]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[0]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[*].name') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[1].name') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[0].not_exist_key') | head 1 | fields res") + + spark.sql("source = myglue_test.default.http_logs | eval res = json_extract('{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}', '$.student[10]') | head 1 | fields res") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > 0) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > -10) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(json_object(\"a\",1,\"b\",-1),json_object(\"a\",-1,\"b\",-1)), result = forall(array, x -> x.a > 0) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(json_object(\"a\",1,\"b\",-1),json_object(\"a\",-1,\"b\",-1)), result = exists(array, x -> x.b < 0) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 0) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 10) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 0) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 10) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,3), result = transform(array, x -> x + 1) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,3), result = transform(array, (x, y) -> x + y) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x) | head 1 | fields result") + + spark.sql("source = myglue_test.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x, acc -> acc * 10) | head 1 | fields result") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eventstats avg(salary) by country | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by country | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as countby span(age, 10) | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by span(age, 10) as age_span, country | sort id | head 10") + + spark.sql("source=myglue_test.default.people | where country != 'USA' | eventstats stddev_samp(salary), stddev_pop(salary), percentile_approx(salary, 60) by span(salary, 1000) as salary_span | sort id | head 10") + + spark.sql("source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age by occupation, country | eventstats avg(avg_age) as avg_state_age by country | sort id | head 10") + + try { + spark.sql("source=myglue_test.default.people | eventstats distinct_count(salary) by span(salary, 1000) as age_span") + throw new Error + } catch { + case e: Exception => null + } + + spark.sql("source = myglue_test.tpch_csv.lineitem| where l_shipdate <= subdate(date('1998-12-01'), 90)| stats sum(l_quantity) as sum_qty, sum(l_extendedprice) as sum_base_price, sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, avg(l_quantity) as avg_qty, avg(l_extendedprice) as avg_price, avg(l_discount) as avg_disc, count() as count_order by l_returnflag, l_linestatus| sort l_returnflag, l_linestatus") + + spark.sql("source = myglue_test.tpch_csv.part| join ON p_partkey = ps_partkey myglue_test.tpch_csv.partsupp| join ON s_suppkey = ps_suppkey myglue_test.tpch_csv.supplier| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation| join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region| where p_size = 15 AND like(p_type, '%BRASS') AND r_name = 'EUROPE' AND ps_supplycost = [ source = myglue_test.tpch_csv.partsupp | join ON s_suppkey = ps_suppkey myglue_test.tpch_csv.supplier | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation | join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region | where r_name = 'EUROPE' | stats MIN(ps_supplycost) ]| sort - s_acctbal, n_name, s_name, p_partkey| head 100") + + spark.sql("source = myglue_test.tpch_csv.customer| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem| where c_mktsegment = 'BUILDING' AND o_orderdate < date('1995-03-15') AND l_shipdate > date('1995-03-15')| stats sum(l_extendedprice * (1 - l_discount)) as revenue by l_orderkey, o_orderdate, o_shippriority | sort - revenue, o_orderdate| head 10") + + spark.sql("source = myglue_test.tpch_csv.orders| where o_orderdate >= date('1993-07-01') and o_orderdate < date_add(date('1993-07-01'), interval 3 month) and exists [ source = myglue_test.tpch_csv.lineitem | where l_orderkey = o_orderkey and l_commitdate < l_receiptdate ]| stats count() as order_count by o_orderpriority| sort o_orderpriority") + + spark.sql("source = myglue_test.tpch_csv.customer| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem| join ON l_suppkey = s_suppkey AND c_nationkey = s_nationkey myglue_test.tpch_csv.supplier| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation| join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region| where r_name = 'ASIA' AND o_orderdate >= date('1994-01-01') AND o_orderdate < date_add(date('1994-01-01'), interval 1 year)| stats sum(l_extendedprice * (1 - l_discount)) as revenue by n_name| sort - revenue") + + spark.sql("source = myglue_test.tpch_csv.lineitem| where l_shipdate >= date('1994-01-01') and l_shipdate < adddate(date('1994-01-01'), 365) and l_discount between .06 - 0.01 and .06 + 0.01 and l_quantity < 24| stats sum(l_extendedprice * l_discount) as revenue") + + spark.sql("source = [ source = myglue_test.tpch_csv.supplier | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.lineitem | join ON o_orderkey = l_orderkey myglue_test.tpch_csv.orders | join ON c_custkey = o_custkey myglue_test.tpch_csv.customer | join ON s_nationkey = n1.n_nationkey myglue_test.tpch_csv.nation as n1 | join ON c_nationkey = n2.n_nationkey myglue_test.tpch_csv.nation as n2 | where l_shipdate between date('1995-01-01') and date('1996-12-31') and n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY' or n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE' | eval supp_nation = n1.n_name, cust_nation = n2.n_name, l_year = year(l_shipdate), volume = l_extendedprice * (1 - l_discount) | fields supp_nation, cust_nation, l_year, volume ] as shipping| stats sum(volume) as revenue by supp_nation, cust_nation, l_year| sort supp_nation, cust_nation, l_year") + + spark.sql("source = [ source = myglue_test.tpch_csv.part | join ON p_partkey = l_partkey myglue_test.tpch_csv.lineitem | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.supplier | join ON l_orderkey = o_orderkey myglue_test.tpch_csv.orders | join ON o_custkey = c_custkey myglue_test.tpch_csv.customer | join ON c_nationkey = n1.n_nationkey myglue_test.tpch_csv.nation as n1 | join ON s_nationkey = n2.n_nationkey myglue_test.tpch_csv.nation as n2 | join ON n1.n_regionkey = r_regionkey myglue_test.tpch_csv.region | where r_name = 'AMERICA' AND p_type = 'ECONOMY ANODIZED STEEL' and o_orderdate between date('1995-01-01') and date('1996-12-31') | eval o_year = year(o_orderdate) | eval volume = l_extendedprice * (1 - l_discount) | eval nation = n2.n_name | fields o_year, volume, nation ] as all_nations| stats sum(case(nation = 'BRAZIL', volume else 0)) as sum_case, sum(volume) as sum_volume by o_year| eval mkt_share = sum_case / sum_volume| fields mkt_share, o_year| sort o_year") + + spark.sql("source = [ source = myglue_test.tpch_csv.part | join ON p_partkey = l_partkey myglue_test.tpch_csv.lineitem | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.supplier | join ON ps_partkey = l_partkey and ps_suppkey = l_suppkey myglue_test.tpch_csv.partsupp | join ON o_orderkey = l_orderkey myglue_test.tpch_csv.orders | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation | where like(p_name, '%green%') | eval nation = n_name | eval o_year = year(o_orderdate) | eval amount = l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity | fields nation, o_year, amount ] as profit| stats sum(amount) as sum_profit by nation, o_year| sort nation, - o_year") + + spark.sql("source = myglue_test.tpch_csv.customer| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem| join ON c_nationkey = n_nationkey myglue_test.tpch_csv.nation| where o_orderdate >= date('1993-10-01') AND o_orderdate < date_add(date('1993-10-01'), interval 3 month) AND l_returnflag = 'R'| stats sum(l_extendedprice * (1 - l_discount)) as revenue by c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment| sort - revenue| head 20") + + spark.sql("source = myglue_test.tpch_csv.partsupp| join ON ps_suppkey = s_suppkey myglue_test.tpch_csv.supplier| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation| where n_name = 'GERMANY'| stats sum(ps_supplycost * ps_availqty) as value by ps_partkey| where value > [ source = myglue_test.tpch_csv.partsupp | join ON ps_suppkey = s_suppkey myglue_test.tpch_csv.supplier | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation | where n_name = 'GERMANY' | stats sum(ps_supplycost * ps_availqty) as check | eval threshold = check * 0.0001000000 | fields threshold ]| sort - value") + + spark.sql("source = myglue_test.tpch_csv.orders| join ON o_orderkey = l_orderkey myglue_test.tpch_csv.lineitem| where l_commitdate < l_receiptdate and l_shipdate < l_commitdate and l_shipmode in ('MAIL', 'SHIP') and l_receiptdate >= date('1994-01-01') and l_receiptdate < date_add(date('1994-01-01'), interval 1 year)| stats sum(case(o_orderpriority = '1-URGENT' or o_orderpriority = '2-HIGH', 1 else 0)) as high_line_count, sum(case(o_orderpriority != '1-URGENT' and o_orderpriority != '2-HIGH', 1 else 0)) as low_line_countby by l_shipmode| sort l_shipmode") + + spark.sql("source = [ source = myglue_test.tpch_csv.customer | left outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') myglue_test.tpch_csv.orders | stats count(o_orderkey) as c_count by c_custkey ] as c_orders| stats count() as custdist by c_count| sort - custdist, - c_count") + + spark.sql("source = myglue_test.tpch_csv.lineitem| join ON l_partkey = p_partkey AND l_shipdate >= date('1995-09-01') AND l_shipdate < date_add(date('1995-09-01'), interval 1 month) myglue_test.tpch_csv.part| stats sum(case(like(p_type, 'PROMO%'), l_extendedprice * (1 - l_discount) else 0)) as sum1, sum(l_extendedprice * (1 - l_discount)) as sum2| eval promo_revenue = 100.00 * sum1 / sum2 // Stats and Eval commands can combine when issues/819 resolved| fields promo_revenue") + + spark.sql("source = myglue_test.tpch_csv.supplier| join right = revenue0 ON s_suppkey = supplier_no [ source = myglue_test.tpch_csv.lineitem | where l_shipdate >= date('1996-01-01') AND l_shipdate < date_add(date('1996-01-01'), interval 3 month) | eval supplier_no = l_suppkey | stats sum(l_extendedprice * (1 - l_discount)) as total_revenue by supplier_no ]| where total_revenue = [ source = [ source = myglue_test.tpch_csv.lineitem | where l_shipdate >= date('1996-01-01') AND l_shipdate < date_add(date('1996-01-01'), interval 3 month) | eval supplier_no = l_suppkey | stats sum(l_extendedprice * (1 - l_discount)) as total_revenue by supplier_no ] | stats max(total_revenue) ]| sort s_suppkey| fields s_suppkey, s_name, s_address, s_phone, total_revenue") + + spark.sql("source = myglue_test.tpch_csv.partsupp| join ON p_partkey = ps_partkey myglue_test.tpch_csv.part| where p_brand != 'Brand#45' and not like(p_type, 'MEDIUM POLISHED%') and p_size in (49, 14, 23, 45, 19, 3, 36, 9) and ps_suppkey not in [ source = myglue_test.tpch_csv.supplier | where like(s_comment, '%Customer%Complaints%') | fields s_suppkey ]| stats distinct_count(ps_suppkey) as supplier_cnt by p_brand, p_type, p_size| sort - supplier_cnt, p_brand, p_type, p_size") + + spark.sql("source = myglue_test.tpch_csv.lineitem| join ON p_partkey = l_partkey myglue_test.tpch_csv.part| where p_brand = 'Brand#23' and p_container = 'MED BOX' and l_quantity < [ source = myglue_test.tpch_csv.lineitem | where l_partkey = p_partkey | stats avg(l_quantity) as avg | eval `0.2 * avg` = 0.2 * avg | fields `0.2 * avg` ]| stats sum(l_extendedprice) as sum| eval avg_yearly = sum / 7.0| fields avg_yearly") + + spark.sql("source = myglue_test.tpch_csv.customer| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders| join ON o_orderkey = l_orderkey myglue_test.tpch_csv.lineitem| where o_orderkey in [ source = myglue_test.tpch_csv.lineitem | stats sum(l_quantity) as sum by l_orderkey | where sum > 300 | fields l_orderkey ]| stats sum(l_quantity) by c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice| sort - o_totalprice, o_orderdate| head 100") + + spark.sql("source = myglue_test.tpch_csv.lineitem| join ON p_partkey = l_partkey and p_brand = 'Brand#12' and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') and l_quantity >= 1 and l_quantity <= 1 + 10 and p_size between 1 and 5 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' OR p_partkey = l_partkey and p_brand = 'Brand#23' and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') and l_quantity >= 10 and l_quantity <= 10 + 10 and p_size between 1 and 10 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' OR p_partkey = l_partkey and p_brand = 'Brand#34' and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') and l_quantity >= 20 and l_quantity <= 20 + 10 and p_size between 1 and 15 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' myglue_test.tpch_csv.part") + + spark.sql("source = myglue_test.tpch_csv.supplier| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation| where n_name = 'CANADA' and s_suppkey in [ source = myglue_test.tpch_csv.partsupp | where ps_partkey in [ source = myglue_test.tpch_csv.part | where like(p_name, 'forest%') | fields p_partkey ] and ps_availqty > [ source = myglue_test.tpch_csv.lineitem | where l_partkey = ps_partkey and l_suppkey = ps_suppkey and l_shipdate >= date('1994-01-01') and l_shipdate < date_add(date('1994-01-01'), interval 1 year) | stats sum(l_quantity) as sum_l_quantity | eval half_sum_l_quantity = 0.5 * sum_l_quantity | fields half_sum_l_quantity ] | fields ps_suppkey ]") + + spark.sql("source = myglue_test.tpch_csv.supplier| join ON s_suppkey = l1.l_suppkey myglue_test.tpch_csv.lineitem as l1| join ON o_orderkey = l1.l_orderkey myglue_test.tpch_csv.orders| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation| where o_orderstatus = 'F' and l1.l_receiptdate > l1.l_commitdate and exists [ source = myglue_test.tpch_csv.lineitem as l2 | where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey != l1.l_suppkey ] and not exists [ source = myglue_test.tpch_csv.lineitem as l3 | where l3.l_orderkey = l1.l_orderkey and l3.l_suppkey != l1.l_suppkey and l3.l_receiptdate > l3.l_commitdate ] and n_name = 'SAUDI ARABIA'| stats count() as numwait by s_name| sort - numwait, s_name| head 100") + + spark.sql("source = [ source = myglue_test.tpch_csv.customer | where substring(c_phone, 1, 2) in ('13', '31', '23', '29', '30', '18', '17') and c_acctbal > [ source = myglue_test.tpch_csv.customer | where c_acctbal > 0.00 and substring(c_phone, 1, 2) in ('13', '31', '23', '29', '30', '18', '17') | stats avg(c_acctbal) ] and not exists [ source = myglue_test.tpch_csv.orders | where o_custkey = c_custkey ] | eval cntrycode = substring(c_phone, 1, 2) | fields cntrycode, c_acctbal ] as custsale| stats count() as numcust, sum(c_acctbal) as totacctbal by cntrycode| sort cntrycode") + +} diff --git a/docker/integ-test/spark-defaults.conf b/docker/integ-test/spark-defaults.conf new file mode 100644 index 000000000..19b9e4ec1 --- /dev/null +++ b/docker/integ-test/spark-defaults.conf @@ -0,0 +1,35 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory +# spark.serializer org.apache.spark.serializer.KryoSerializer +# spark.driver.memory 5g +# spark.executor.extraJavaOptions -XX:+PrintGCDetails -Dkey=value -Dnumbers="one two three" +spark.sql.extensions org.opensearch.flint.spark.FlintPPLSparkExtensions,org.opensearch.flint.spark.FlintSparkExtensions +spark.sql.catalog.dev org.apache.spark.opensearch.catalog.OpenSearchCatalog +spark.datasource.flint.host opensearch +spark.datasource.flint.port 9200 +spark.datasource.flint.scheme http +spark.datasource.flint.auth basic +spark.datasource.flint.auth.username admin +spark.datasource.flint.auth.password C0rrecthorsebatterystaple. diff --git a/docker/integ-test/spark-master-entrypoint.sh b/docker/integ-test/spark-master-entrypoint.sh new file mode 100755 index 000000000..a21c20643 --- /dev/null +++ b/docker/integ-test/spark-master-entrypoint.sh @@ -0,0 +1,17 @@ +#!/bin/bash + +function start_spark_connect() { + sc_version=$(ls -1 /opt/bitnami/spark/jars/spark-core_*.jar | sed -e 's/^.*\/spark-core_//' -e 's/\.jar$//' -e 's/-/:/') + + attempt=1 + while [ -e "/tmp/spark_master_running" -a "$attempt" -le 10 ]; do + sleep 1 + /opt/bitnami/spark/sbin/start-connect-server.sh --master spark://spark:7077 --packages org.apache.spark:spark-connect_${sc_version} + attempt=$(($attempt+1)) + done +} + +touch /tmp/spark_master_running +start_spark_connect & +/opt/bitnami/scripts/spark/entrypoint.sh /opt/bitnami/scripts/spark/run.sh +rm /tmp/spark_master_running diff --git a/integ-test/script/README.md b/integ-test/script/README.md index 7ce0c6886..f9e9a8e93 100644 --- a/integ-test/script/README.md +++ b/integ-test/script/README.md @@ -17,41 +17,55 @@ Apart from the basic feature, it also has some advanced functionality includes: ### Usage To use this script, you need to have Python **3.6** or higher installed. It also requires the following Python libraries: ```shell -pip install requests pandas openpyxl +pip install requests pandas openpyxl pyspark setuptools pyarrow grpcio grpcio-status protobuf +``` + +Build the Flint and PPL extensions for Spark. +``` +sbt clean +sbt sparkSqlApplicationCosmetic/assembly sparkPPLCosmetic/assembly +``` + +Next start the Docker containers that will be used for the tests. In the directory `docker/integ-test` +```shell +docker compose up -d +``` + +After the tests are finished, the Docker containers can be stopped from the directory `docker/integ-test` with: +```shell +docker compose down ``` After getting the requisite libraries, you can run the script with the following command line parameters in your shell: ```shell -python SanityTest.py --base-url ${URL_ADDRESS} --username *** --password *** --datasource ${DATASOURCE_NAME} --input-csv test_cases.csv --output-file test_report --max-workers 2 --check-interval 10 --timeout 600 +python SanityTest.py --spark-url ${SPARK_URL} --username *** --password *** --opensearch-url ${OPENSEARCH_URL} --input-csv test_cases.csv --output-file test_report ``` -You need to replace the placeholders with your actual values of URL_ADDRESS, DATASOURCE_NAME and USERNAME, PASSWORD for authentication to your endpoint. +You need to replace the placeholders with your actual values of SPARK_URL, OPENSEARCH_URL and USERNAME, PASSWORD for authentication to your endpoint. + +Running against the docker cluster, `SPARK_URL` should be set to `sc://localhost:15002` and `OPENSEARCH_URL` should be set +to `http://localhost:9200` For more details of the command line parameters, you can see the help manual via command: ```shell python SanityTest.py --help -usage: SanityTest.py [-h] --base-url BASE_URL --username USERNAME --password PASSWORD --datasource DATASOURCE --input-csv INPUT_CSV - --output-file OUTPUT_FILE [--max-workers MAX_WORKERS] [--check-interval CHECK_INTERVAL] [--timeout TIMEOUT] +usage: SanityTest.py [-h] --spark-url SPARK_URL --username USERNAME --password PASSWORD --datasource DATASOURCE --input-csv INPUT_CSV + --output-file OPENSEARCH_URL [--max-workers MAX_WORKERS] [--check-interval CHECK_INTERVAL] [--timeout TIMEOUT] [--start-row START_ROW] [--end-row END_ROW] Run tests from a CSV file and generate a report. options: -h, --help show this help message and exit - --base-url BASE_URL Base URL of the service + --spark-url SPARK_URL Spark Connect URL of the service --username USERNAME Username for authentication --password PASSWORD Password for authentication - --datasource DATASOURCE - Datasource name + --output-file OPENSEARCH_URL + URL of the OpenSearch service --input-csv INPUT_CSV Path to the CSV file containing test queries --output-file OUTPUT_FILE Path to the output report file - --max-workers MAX_WORKERS - optional, Maximum number of worker threads (default: 2) - --check-interval CHECK_INTERVAL - optional, Check interval in seconds (default: 10) - --timeout TIMEOUT optional, Timeout in seconds (default: 600) --start-row START_ROW optional, The start row of the query to run, start from 1 --end-row END_ROW optional, The end row of the query to run, not included @@ -64,7 +78,20 @@ As claimed in the description, the input CSV file should at least have the colum We also provide a sample input CSV file `test_cases.csv` for reference. It includes all sanity test cases we have currently in the Flint. -**TODO**: the prerequisite data of the test cases and ingesting process +### Indices and Data for Testing +After the docker containers have started, the test script will try to create indices that are needed for testing. It will look in the directory `data`. It will start by +looking for all files with names ending with `.mapping.json`. The start of the filename is the name of the index to create. The contents of the file is the field mappings. + +[Supported field types](https://opensearch.org/docs/latest/field-types/supported-field-types/index/) + +[Example mapping](https://opensearch.org/docs/latest/field-types/supported-field-types/index/#example) + +After the indices have been created, the script will look for all other files ending with `.json`. These are the files for bulk inserting data into the indices. The start +of the filename is the index to insert data into. The contents of the file are used as the body of the bulk insert request. + +[Bulk Insert](https://opensearch.org/docs/latest/api-reference/document-apis/bulk/) + +[Example Body](https://opensearch.org/docs/latest/api-reference/document-apis/bulk/) ### Report Explanation The generated report contains two files: @@ -78,12 +105,12 @@ It also provides the query_id, session_id and start/end time for each query, whi An example of Excel report: -| query_name | query | expected_status | status | check_status | error | result | Duration (s) | query_id | session_id | Start Time | End Time | -|------------|------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------|---------|--------------|------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|-------------------------------|------------------------------|----------------------|---------------------| -| 1 | describe myglue_test.default.http_logs | SUCCESS | SUCCESS | TRUE | | {'status': 'SUCCESS', 'schema': [{...}, ...], 'datarows': [[...], ...], 'total': 31, 'size': 31} | 37.51 | SHFEVWxDNnZjem15Z2x1ZV90ZXN0 | RkgzZm0xNlA5MG15Z2x1ZV90ZXN0 | 2024-11-07 13:34:10 | 2024-11-07 13:34:47 | -| 2 | source = myglue_test.default.http_logs \| dedup status CONSECUTIVE=true | SUCCESS | FAILED | FALSE | {"Message":"Fail to run query. Cause: Consecutive deduplication is not supported"} | | 39.53 | dVNlaVVxOFZrZW15Z2x1ZV90ZXN0 | ZGU2MllVYmI4dG15Z2x1ZV90ZXN0 | 2024-11-07 13:34:10 | 2024-11-07 13:34:49 | -| 3 | source = myglue_test.default.http_logs \| eval res = json_keys(json('{"account_number":1,"balance":39225,"age":32,"gender":"M"}')) \| head 1 \| fields res | SUCCESS | SUCCESS | TRUE | | {'status': 'SUCCESS', 'schema': [{'name': 'res', 'type': 'array'}], 'datarows': [[['account_number', 'balance', 'age', 'gender']]], 'total': 1, 'size': 1} | 12.77 | WHQxaXlVSGtGUm15Z2x1ZV90ZXN0 | RkgzZm0xNlA5MG15Z2x1ZV90ZXN0 | 2024-11-07 13:34:47 | 2024-11-07 13:38:45 | -| ... | ... | ... | ... | ... | | | ... | ... | ... | ... | ... | +| query_name | query | expected_status | status | check_status | error | result | duration (s) | Start Time | End Time | +|------------|------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------|---------|--------------|------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|----------------------|---------------------| +| 1 | describe myglue_test.default.http_logs | SUCCESS | SUCCESS | TRUE | | {'status': 'SUCCESS', 'schema': [{...}, ...], 'datarows': [[...], ...], 'total': 31, 'size': 31} | 37.51 | 2024-11-07 13:34:10 | 2024-11-07 13:34:47 | +| 2 | source = myglue_test.default.http_logs \| dedup status CONSECUTIVE=true | SUCCESS | FAILED | FALSE | {"Message":"Fail to run query. Cause: Consecutive deduplication is not supported"} | | 39.53 | 2024-11-07 13:34:10 | 2024-11-07 13:34:49 | +| 3 | source = myglue_test.default.http_logs \| eval res = json_keys(json('{"account_number":1,"balance":39225,"age":32,"gender":"M"}')) \| head 1 \| fields res | SUCCESS | SUCCESS | TRUE | | {'status': 'SUCCESS', 'schema': [{'name': 'res', 'type': 'array'}], 'datarows': [[['account_number', 'balance', 'age', 'gender']]], 'total': 1, 'size': 1} | 12.77 | 2024-11-07 13:34:47 | 2024-11-07 13:38:45 | +| ... | ... | ... | ... | ... | | | ... | ... | ... | #### JSON Report @@ -103,7 +130,7 @@ An example of JSON report: "detailed_results": [ { "query_name": 1, - "query": "source = myglue_test.default.http_logs | stats avg(size)", + "query": "source = dev.default.http_logs | stats avg(size)", "query_id": "eFZmTlpTa3EyTW15Z2x1ZV90ZXN0", "session_id": "bFJDMWxzb2NVUm15Z2x1ZV90ZXN0", "status": "SUCCESS", @@ -130,7 +157,7 @@ An example of JSON report: }, { "query_name": 2, - "query": "source = myglue_test.default.http_logs | eval res = json_keys(json(\u2018{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}')) | head 1 | fields res", + "query": "source = def.default.http_logs | eval res = json_keys(json(\u2018{\"teacher\":\"Alice\",\"student\":[{\"name\":\"Bob\",\"rank\":1},{\"name\":\"Charlie\",\"rank\":2}]}')) | head 1 | fields res", "query_id": "bjF4Y1VnbXdFYm15Z2x1ZV90ZXN0", "session_id": "c3pvU1V6OW8xM215Z2x1ZV90ZXN0", "status": "FAILED", @@ -142,7 +169,7 @@ An example of JSON report: }, { "query_name": 2, - "query": "source = myglue_test.default.http_logs | eval col1 = size, col2 = clientip | stats avg(col1) by col2", + "query": "source = dev.default.http_logs | eval col1 = size, col2 = clientip | stats avg(col1) by col2", "query_id": "azVyMFFORnBFRW15Z2x1ZV90ZXN0", "session_id": "VWF0SEtrNWM3bm15Z2x1ZV90ZXN0", "status": "TIMEOUT", diff --git a/integ-test/script/SanityTest.py b/integ-test/script/SanityTest.py index eb97752b4..b4e6210bb 100644 --- a/integ-test/script/SanityTest.py +++ b/integ-test/script/SanityTest.py @@ -3,6 +3,7 @@ SPDX-License-Identifier: Apache-2.0 """ +import glob import signal import sys import requests @@ -11,18 +12,18 @@ import time import logging from datetime import datetime -import pandas as pd import argparse from requests.auth import HTTPBasicAuth -from concurrent.futures import ThreadPoolExecutor, as_completed +from pyspark.sql import SparkSession import threading +import pandas as pd """ Environment: python3 Example to use this script: -python SanityTest.py --base-url ${URL_ADDRESS} --username *** --password *** --datasource ${DATASOURCE_NAME} --input-csv test_queries.csv --output-file test_report --max-workers 2 --check-interval 10 --timeout 600 +python SanityTest.py --spark-url ${SPARK_URL} --username *** --password *** --opensearch_url ${OPENSEARCH_URL} --input-csv test_queries.csv --output-file test_report The input file test_queries.csv should contain column: `query` @@ -33,24 +34,19 @@ """ class FlintTester: - def __init__(self, base_url, username, password, datasource, max_workers, check_interval, timeout, output_file, start_row, end_row, log_level): - self.base_url = base_url + def __init__(self, spark_url, username, password, opensearch_url, output_file, start_row, end_row, log_level): + self.spark_url = spark_url self.auth = HTTPBasicAuth(username, password) - self.datasource = datasource - self.headers = { 'Content-Type': 'application/json' } - self.max_workers = max_workers - self.check_interval = check_interval - self.timeout = timeout + self.opensearch_url = opensearch_url self.output_file = output_file self.start = start_row - 1 if start_row else None self.end = end_row - 1 if end_row else None self.log_level = log_level - self.max_attempts = (int)(timeout / check_interval) self.logger = self._setup_logger() - self.executor = ThreadPoolExecutor(max_workers=self.max_workers) - self.thread_local = threading.local() self.test_results = [] + self.spark_client = SparkSession.builder.remote(spark_url).appName("integ-test").getOrCreate() + def _setup_logger(self): logger = logging.getLogger('FlintTester') logger.setLevel(self.log_level) @@ -72,126 +68,80 @@ def _setup_logger(self): return logger + # Create the indices needed for the tests + def create_indices(self): + self.logger.info("Creating indices") - def get_session_id(self): - if not hasattr(self.thread_local, 'session_id'): - self.thread_local.session_id = "empty_session_id" - self.logger.debug(f"get session id {self.thread_local.session_id}") - return self.thread_local.session_id + json_files = glob.glob('data/*.json') + mapping_files = [f for f in json_files if f.endswith('.mapping.json')] + data_files = [f for f in json_files if not f.endswith('.mapping.json')] + existing_indices = set() - def set_session_id(self, session_id): - """Reuse the session id for the same thread""" - self.logger.debug(f"set session id {session_id}") - self.thread_local.session_id = session_id + for mapping_file in mapping_files: + index_name = mapping_file[5 : mapping_file.index('.')] - # Call submit API to submit the query - def submit_query(self, query, session_id="Empty"): - url = f"{self.base_url}/_plugins/_async_query" - payload = { - "datasource": self.datasource, - "lang": "ppl", - "query": query, - "sessionId": session_id - } - self.logger.debug(f"Submit query with payload: {payload}") - response_json = None - try: - response = requests.post(url, auth=self.auth, json=payload, headers=self.headers) - response_json = response.json() - response.raise_for_status() - return response_json - except Exception as e: - return {"error": f"{str(e)}, got response {response_json}"} + self.logger.info(f"Checking if index exists: {index_name}") + response = requests.get(f'{self.opensearch_url}/{index_name}', auth=self.auth) + if response.status_code == 200: + existing_indices.add(index_name) + continue - # Call get API to check the query status - def get_query_result(self, query_id): - url = f"{self.base_url}/_plugins/_async_query/{query_id}" - response_json = None - try: - response = requests.get(url, auth=self.auth) - response_json = response.json() - response.raise_for_status() - return response_json - except Exception as e: - return {"status": "FAILED", "error": f"{str(e)}, got response {response_json}"} + self.logger.info(f"Creating index: {index_name}") - # Call delete API to cancel the query - def cancel_query(self, query_id): - url = f"{self.base_url}/_plugins/_async_query/{query_id}" - response_json = None - try: - response = requests.delete(url, auth=self.auth) - response_json = response.json() - response.raise_for_status() - self.logger.info(f"Cancelled query [{query_id}] with info {response.json()}") - return response_json - except Exception as e: - self.logger.warning(f"Cancel query [{query_id}] error: {str(e)}, got response {response_json}") + file_data = open(mapping_file, 'rb').read() + headers = {'Content-Type': 'application/json'} + + response = requests.put(f'{self.opensearch_url}/{index_name}', auth=self.auth, headers=headers, data=file_data) + if response.status_code != 200: + self.logger.error(f'Failed to create index: {index_name}') + response.raise_for_status() + + for data_file in data_files: + index_name = data_file[5 : data_file.index('.')] + if index_name in existing_indices: + continue + + self.logger.info(f"Populating index: {index_name}") + + file_data = open(data_file, 'rb').read() + headers = {'Content-Type': 'application/x-ndjson'} + + response = requests.post(f'{self.opensearch_url}/{index_name}/_bulk', auth=self.auth, headers=headers, data=file_data) + if response.status_code != 200: + response.raise_for_status() # Run the test and return the result def run_test(self, query, seq_id, expected_status): self.logger.info(f"Starting test: {seq_id}, {query}") start_time = datetime.now() - pre_session_id = self.get_session_id() - submit_result = self.submit_query(query, pre_session_id) - if "error" in submit_result: - self.logger.warning(f"Submit error: {submit_result}") - return { - "query_name": seq_id, - "query": query, - "expected_status": expected_status, - "status": "SUBMIT_FAILED", - "check_status": "SUBMIT_FAILED" == expected_status if expected_status else None, - "error": submit_result["error"], - "duration": 0, - "start_time": start_time, - "end_time": datetime.now() - } - - query_id = submit_result["queryId"] - session_id = submit_result["sessionId"] - self.logger.info(f"Submit return: {submit_result}") - if (session_id != pre_session_id): - self.logger.info(f"Update session id from {pre_session_id} to {session_id}") - self.set_session_id(session_id) - - test_result = self.check_query_status(query_id) + + query_str = query.replace('\n', ' ') + status = None + result = None + error_str = None + try: + result = self.spark_client.sql(query_str) + status = 'SUCCESS' + except Exception as e: + status = 'FAILED' + error_str = str(e) + end_time = datetime.now() duration = (end_time - start_time).total_seconds() return { "query_name": seq_id, "query": query, - "query_id": query_id, - "session_id": session_id, "expected_status": expected_status, - "status": test_result["status"], - "check_status": test_result["status"] == expected_status if expected_status else None, - "error": test_result.get("error", ""), - "result": test_result if test_result["status"] == "SUCCESS" else None, + "status": status, + "check_status": status == expected_status if expected_status else None, + "error": error_str if error_str else None, + "result": result, "duration": duration, "start_time": start_time, "end_time": end_time } - # Check the status of the query periodically until it is completed or failed or exceeded the timeout - def check_query_status(self, query_id): - query_id = query_id - - for attempt in range(self.max_attempts): - time.sleep(self.check_interval) - result = self.get_query_result(query_id) - - if result["status"] == "FAILED" or result["status"] == "SUCCESS": - return result - - # Cancel the query if it exceeds the timeout - self.cancel_query(query_id) - return { - "status": "TIMEOUT", - "error": "Query execution exceeded " + str(self.timeout) + " seconds with last status: " + result["status"], - } - def run_tests_from_csv(self, csv_file): with open(csv_file, 'r') as f: reader = csv.DictReader(f) @@ -200,20 +150,15 @@ def run_tests_from_csv(self, csv_file): # Filtering queries based on start and end queries = queries[self.start:self.end] - # Parallel execution - futures = [self.executor.submit(self.run_test, query, seq_id, expected_status) for query, seq_id, expected_status in queries] - for future in as_completed(futures): - result = future.result() - self.logger.info(f"Completed test: {result["query_name"]}, {result["query"]}, got result status: {result["status"]}") - self.test_results.append(result) + self.test_results = [] + for query in queries: + self.test_results.append(self.run_test(query[0], query[1], query[2])) def generate_report(self): self.logger.info("Generating report...") total_queries = len(self.test_results) successful_queries = sum(1 for r in self.test_results if r['status'] == 'SUCCESS') failed_queries = sum(1 for r in self.test_results if r['status'] == 'FAILED') - submit_failed_queries = sum(1 for r in self.test_results if r['status'] == 'SUBMIT_FAILED') - timeout_queries = sum(1 for r in self.test_results if r['status'] == 'TIMEOUT') # Create report report = { @@ -221,8 +166,6 @@ def generate_report(self): "total_queries": total_queries, "successful_queries": successful_queries, "failed_queries": failed_queries, - "submit_failed_queries": submit_failed_queries, - "timeout_queries": timeout_queries, "execution_time": sum(r['duration'] for r in self.test_results) }, "detailed_results": self.test_results @@ -249,15 +192,12 @@ def signal_handler(sig, frame, tester): def main(): # Parse command line arguments parser = argparse.ArgumentParser(description="Run tests from a CSV file and generate a report.") - parser.add_argument("--base-url", required=True, help="Base URL of the service") + parser.add_argument("--spark-url", required=True, help="URL of the Spark service") parser.add_argument("--username", required=True, help="Username for authentication") parser.add_argument("--password", required=True, help="Password for authentication") - parser.add_argument("--datasource", required=True, help="Datasource name") + parser.add_argument("--opensearch-url", required=True, help="URL of the OpenSearch service") parser.add_argument("--input-csv", required=True, help="Path to the CSV file containing test queries") parser.add_argument("--output-file", required=True, help="Path to the output report file") - parser.add_argument("--max-workers", type=int, default=2, help="optional, Maximum number of worker threads (default: 2)") - parser.add_argument("--check-interval", type=int, default=5, help="optional, Check interval in seconds (default: 5)") - parser.add_argument("--timeout", type=int, default=600, help="optional, Timeout in seconds (default: 600)") parser.add_argument("--start-row", type=int, default=None, help="optional, The start row of the query to run, start from 1") parser.add_argument("--end-row", type=int, default=None, help="optional, The end row of the query to run, not included") parser.add_argument("--log-level", default="INFO", help="optional, Log level (DEBUG, INFO, WARNING, ERROR, CRITICAL, default: INFO)") @@ -265,13 +205,10 @@ def main(): args = parser.parse_args() tester = FlintTester( - base_url=args.base_url, + spark_url=args.spark_url, username=args.username, password=args.password, - datasource=args.datasource, - max_workers=args.max_workers, - check_interval=args.check_interval, - timeout=args.timeout, + opensearch_url=args.opensearch_url, output_file=args.output_file, start_row=args.start_row, end_row=args.end_row, @@ -282,6 +219,9 @@ def main(): signal.signal(signal.SIGINT, lambda sig, frame: signal_handler(sig, frame, tester)) signal.signal(signal.SIGTERM, lambda sig, frame: signal_handler(sig, frame, tester)) + # Create indices + tester.create_indices() + # Running tests tester.run_tests_from_csv(args.input_csv) diff --git a/integ-test/script/data/customer.mapping.json b/integ-test/script/data/customer.mapping.json new file mode 100644 index 000000000..a98d473a2 --- /dev/null +++ b/integ-test/script/data/customer.mapping.json @@ -0,0 +1,30 @@ +{ + "mappings": { + "properties": { + "c_custkey": { + "type": "integer" + }, + "c_name": { + "type": "text" + }, + "c_address": { + "type": "text" + }, + "c_nationkey": { + "type": "integer" + }, + "c_phone": { + "type": "text" + }, + "c_acctbal": { + "type": "double" + }, + "c_mktsegment": { + "type": "text" + }, + "c_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/http_logs.json b/integ-test/script/data/http_logs.json new file mode 100644 index 000000000..ff2aa2fca --- /dev/null +++ b/integ-test/script/data/http_logs.json @@ -0,0 +1,12 @@ +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696154400000, "year": 2023, "month": 10, "day": 1, "clientip": "40.135.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696154700000, "year": 2023, "month": 10, "day": 1, "clientip": "232.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696155000000, "year": 2023, "month": 10, "day": 1, "clientip": "26.1.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696155300000, "year": 2023, "month": 10, "day": 1, "clientip": "247.37.0.0", "request": "GET /french/splash_inet.html HTTP/1.0", "status": 200, "size": 3781} +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696155600000, "year": 2023, "month": 10, "day": 1, "clientip": "247.37.0.0", "request": "GET /images/hm_nbg.jpg HTTP/1.0", "status": 304, "size": 0} +{"index": {"_index": "http_logs"}} +{"@timestamp": 1696155900000, "year": 2023, "month": 10, "day": 1, "clientip": "252.0.0.0", "request": "GET /images/hm_bg.jpg HTTP/1.0", "status": 200, "size": 24736} diff --git a/integ-test/script/data/http_logs.mapping.json b/integ-test/script/data/http_logs.mapping.json new file mode 100644 index 000000000..b944fbd4b --- /dev/null +++ b/integ-test/script/data/http_logs.mapping.json @@ -0,0 +1,30 @@ +{ + "mappings": { + "properties": { + "@timestamp": { + "type": "date" + }, + "year": { + "type": "integer" + }, + "month": { + "type": "integer" + }, + "day": { + "type": "integer" + }, + "clientip": { + "type": "keyword" + }, + "request": { + "type": "text" + }, + "status": { + "type": "integer" + }, + "size": { + "type": "integer" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/lineitem.mapping.json b/integ-test/script/data/lineitem.mapping.json new file mode 100644 index 000000000..2fb1cdb40 --- /dev/null +++ b/integ-test/script/data/lineitem.mapping.json @@ -0,0 +1,54 @@ +{ + "mappings": { + "properties": { + "l_orderkey": { + "type": "integer" + }, + "l_partkey": { + "type": "text" + }, + "l_suppkey": { + "type": "integer" + }, + "l_linenumber": { + "type": "integer" + }, + "l_quantity": { + "type": "double" + }, + "l_extendedprice": { + "type": "double" + }, + "l_discount": { + "type": "double" + }, + "l_tax": { + "type": "double" + }, + "l_returnflag": { + "type": "text" + }, + "l_linestatus": { + "type": "text" + }, + "l_shipdate": { + "type": "date" + }, + "l_commitdate": { + "type": "date" + }, + "l_receiptdate": { + "type": "date" + }, + "l_shipinstruct": { + "type": "text" + }, + "l_shipmode": { + "type": "text" + }, + "l_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/nation.mapping.json b/integ-test/script/data/nation.mapping.json new file mode 100644 index 000000000..d0e82e559 --- /dev/null +++ b/integ-test/script/data/nation.mapping.json @@ -0,0 +1,18 @@ +{ + "mappings": { + "properties": { + "n_nationkey": { + "type": "integer" + }, + "n_name": { + "type": "text" + }, + "n_regionkey": { + "type": "integer" + }, + "n_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/nested.json b/integ-test/script/data/nested.json new file mode 100644 index 000000000..eb8af683b --- /dev/null +++ b/integ-test/script/data/nested.json @@ -0,0 +1,10 @@ +{"index": {"_index": "nested"}} +{"int_col": 30, "struct_col": {"field1": {"subfield": "value1"}, "field2": 123}, "struct_col2": {"field1": {"subfield": "valueA"}, "field2": 23}} +{"index": {"_index": "nested"}} +{"int_col": 40, "struct_col": {"field1": {"subfield": "value5"}, "field2": 123}, "struct_col2": {"field1": {"subfield": "valueB"}, "field2": 33}} +{"index": {"_index": "nested"}} +{"int_col": 30, "struct_col": {"field1": {"subfield": "value4"}, "field2": 823}, "struct_col2": {"field1": {"subfield": "valueC"}, "field2": 83}} +{"index": {"_index": "nested"}} +{"int_col": 40, "struct_col": {"field1": {"subfield": "value2"}, "field2": 456}, "struct_col2": {"field1": {"subfield": "valueD"}, "field2": 46}} +{"index": {"_index": "nested"}} +{"int_col": 50, "struct_col": {"field1": {"subfield": "value3"}, "field2": 789}, "struct_col2": {"field1": {"subfield": "valueE"}, "field2": 89}} diff --git a/integ-test/script/data/nested.mapping.json b/integ-test/script/data/nested.mapping.json new file mode 100644 index 000000000..1aa189415 --- /dev/null +++ b/integ-test/script/data/nested.mapping.json @@ -0,0 +1,37 @@ +{ + "mappings": { + "properties": { + "int_col": { + "type": "integer" + }, + "struct_col": { + "properties": { + "field1": { + "properties": { + "subfield": { + "type": "text" + } + } + }, + "field2": { + "type": "integer" + } + } + }, + "struct_col2": { + "properties": { + "field1": { + "properties": { + "subfield": { + "type": "text" + } + } + }, + "field2": { + "type": "integer" + } + } + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/orders.mapping.json b/integ-test/script/data/orders.mapping.json new file mode 100644 index 000000000..59b3cecdd --- /dev/null +++ b/integ-test/script/data/orders.mapping.json @@ -0,0 +1,33 @@ +{ + "mappings": { + "properties": { + "o_orderkey": { + "type": "integer" + }, + "o_custkey": { + "type": "integer" + }, + "o_orderstatus": { + "type": "text" + }, + "o_totalprice": { + "type": "double" + }, + "o_orderdate": { + "type": "date" + }, + "o_orderpriority": { + "type": "text" + }, + "o_clerk": { + "type": "text" + }, + "o_shippriority": { + "type": "integer" + }, + "o_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/part.mapping.json b/integ-test/script/data/part.mapping.json new file mode 100644 index 000000000..8be7e9aa0 --- /dev/null +++ b/integ-test/script/data/part.mapping.json @@ -0,0 +1,33 @@ +{ + "mappings": { + "properties": { + "p_partkey": { + "type": "integer" + }, + "p_name": { + "type": "text" + }, + "p_mfgr": { + "type": "text" + }, + "p_brand": { + "type": "text" + }, + "p_type": { + "type": "text" + }, + "p_size": { + "type": "integer" + }, + "p_container": { + "type": "text" + }, + "p_retailprice": { + "type": "double" + }, + "p_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/partsupp.mapping.json b/integ-test/script/data/partsupp.mapping.json new file mode 100644 index 000000000..13509ad46 --- /dev/null +++ b/integ-test/script/data/partsupp.mapping.json @@ -0,0 +1,21 @@ +{ + "mappings": { + "properties": { + "ps_partkey": { + "type": "integer" + }, + "ps_suppkey": { + "type": "integer" + }, + "ps_availqty": { + "type": "integer" + }, + "ps_supplycost": { + "type": "double" + }, + "ps_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/people.json b/integ-test/script/data/people.json new file mode 100644 index 000000000..4563a2c4b --- /dev/null +++ b/integ-test/script/data/people.json @@ -0,0 +1,12 @@ +{"index": {"_index": "people"}} +{"@timestamp": 1718458823000, "id": 1000, "name": "Jake", "occupation": "Engineer", "country": "England", "salary": 100000} +{"index": {"_index": "people"}} +{"@timestamp": 1718458833000, "id": 1001, "name": "Hello", "occupation": "Artist", "country": "USA", "salary": 70000} +{"index": {"_index": "people"}} +{"@timestamp": 1718458843000, "id": 1002, "name": "John", "occupation": "Doctor", "country": "Canada", "salary": 120000} +{"index": {"_index": "people"}} +{"@timestamp": 1718458853000, "id": 1003, "name": "David", "occupation": "Doctor", "country": null, "salary": 120000} +{"index": {"_index": "people"}} +{"@timestamp": 1718458863000, "id": 1004, "name": "David", "occupation": null, "country": "Canada", "salary": 0} +{"index": {"_index": "people"}} +{"@timestamp": 1718458873000, "id": 1005, "name": "Jane", "occupation": "Scientist", "country": "Canada", "salary": 90000} diff --git a/integ-test/script/data/people.mapping.json b/integ-test/script/data/people.mapping.json new file mode 100644 index 000000000..b5dde8ff6 --- /dev/null +++ b/integ-test/script/data/people.mapping.json @@ -0,0 +1,24 @@ +{ + "mappings": { + "properties": { + "@timestamp": { + "type": "date" + }, + "id": { + "type": "integer" + }, + "name": { + "type": "text" + }, + "occupation": { + "type": "text" + }, + "country": { + "type": "text" + }, + "salary": { + "type": "integer" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/region.mapping.json b/integ-test/script/data/region.mapping.json new file mode 100644 index 000000000..3dddbc580 --- /dev/null +++ b/integ-test/script/data/region.mapping.json @@ -0,0 +1,15 @@ +{ + "mappings": { + "properties": { + "r_regionkey": { + "type": "integer" + }, + "r_name": { + "type": "text" + }, + "r_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/supplier.mapping.json b/integ-test/script/data/supplier.mapping.json new file mode 100644 index 000000000..bdcb933b6 --- /dev/null +++ b/integ-test/script/data/supplier.mapping.json @@ -0,0 +1,27 @@ +{ + "mappings": { + "properties": { + "s_suppkey": { + "type": "integer" + }, + "s_name": { + "type": "text" + }, + "s_address": { + "type": "text" + }, + "s_nationkey": { + "type": "integer" + }, + "s_phone": { + "type": "text" + }, + "s_acctbal": { + "type": "double" + }, + "s_comment": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/data/work_info.json b/integ-test/script/data/work_info.json new file mode 100644 index 000000000..64802bdad --- /dev/null +++ b/integ-test/script/data/work_info.json @@ -0,0 +1,10 @@ +{"index": {"_index": "work_info"}} +{"uid": 1000, "name": "Jake", "department": "IT", "occupation": "Engineer"} +{"index": {"_index": "work_info"}} +{"uid": 1002, "name": "John", "department": "DATA", "occupation": "Scientist"} +{"index": {"_index": "work_info"}} +{"uid": 1003, "name": "David", "department": "HR", "occupation": "Doctor"} +{"index": {"_index": "work_info"}} +{"uid": 1005, "name": "Jane", "department": "DATA", "occupation": "Engineer"} +{"index": {"_index": "work_info"}} +{"uid": 1006, "name": "Tom", "department": "SALES", "occupation": "Artist"} diff --git a/integ-test/script/data/work_info.mapping.json b/integ-test/script/data/work_info.mapping.json new file mode 100644 index 000000000..3fb5e2c28 --- /dev/null +++ b/integ-test/script/data/work_info.mapping.json @@ -0,0 +1,18 @@ +{ + "mappings": { + "properties": { + "uid": { + "type": "integer" + }, + "name": { + "type": "text" + }, + "department": { + "type": "text" + }, + "occupation": { + "type": "text" + } + } + } +} \ No newline at end of file diff --git a/integ-test/script/test_cases.csv b/integ-test/script/test_cases.csv index 7df05f5a3..91500efea 100644 --- a/integ-test/script/test_cases.csv +++ b/integ-test/script/test_cases.csv @@ -1,116 +1,116 @@ query,expected_status -describe myglue_test.default.http_logs,FAILED -describe `myglue_test`.`default`.`http_logs`,FAILED -"source = myglue_test.default.http_logs | dedup 1 status | fields @timestamp, clientip, status, size | head 10",SUCCESS -"source = myglue_test.default.http_logs | dedup status, size | head 10",SUCCESS -source = myglue_test.default.http_logs | dedup 1 status keepempty=true | head 10,SUCCESS -"source = myglue_test.default.http_logs | dedup status, size keepempty=true | head 10",SUCCESS -source = myglue_test.default.http_logs | dedup 2 status | head 10,SUCCESS -"source = myglue_test.default.http_logs | dedup 2 status, size | head 10",SUCCESS -"source = myglue_test.default.http_logs | dedup 2 status, size keepempty=true | head 10",SUCCESS -source = myglue_test.default.http_logs | dedup status CONSECUTIVE=true | fields status,FAILED -"source = myglue_test.default.http_logs | dedup 2 status, size CONSECUTIVE=true | fields status",FAILED -"source = myglue_test.default.http_logs | sort stat | fields @timestamp, clientip, status | head 10",SUCCESS -"source = myglue_test.default.http_logs | fields @timestamp, notexisted | head 10",FAILED -"source = myglue_test.default.nested | fields int_col, struct_col.field1, struct_col2.field1 | head 10",FAILED -"source = myglue_test.default.nested | where struct_col2.field1.subfield > 'valueA' | sort int_col | fields int_col, struct_col.field1.subfield, struct_col2.field1.subfield",FAILED -"source = myglue_test.default.http_logs | fields - @timestamp, clientip, status | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval new_time = @timestamp, new_clientip = clientip | fields - new_time, new_clientip, status | head 10",SUCCESS -source = myglue_test.default.http_logs | eval new_clientip = lower(clientip) | fields - new_clientip | head 10,SUCCESS -"source = myglue_test.default.http_logs | fields + @timestamp, clientip, status | fields - clientip, status | head 10",SUCCESS -"source = myglue_test.default.http_logs | fields - clientip, status | fields + @timestamp, clientip, status| head 10",SUCCESS -source = myglue_test.default.http_logs | where status = 200 | head 10,SUCCESS -source = myglue_test.default.http_logs | where status != 200 | head 10,SUCCESS -source = myglue_test.default.http_logs | where size > 0 | head 10,SUCCESS -source = myglue_test.default.http_logs | where size <= 0 | head 10,SUCCESS -source = myglue_test.default.http_logs | where clientip = '236.14.2.0' | head 10,SUCCESS -source = myglue_test.default.http_logs | where size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100,SUCCESS -"source = myglue_test.default.http_logs | where size <= 0 AND like(request, 'GET%') | head 10",SUCCESS -source = myglue_test.default.http_logs status = 200 | head 10,SUCCESS -source = myglue_test.default.http_logs size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100,SUCCESS -"source = myglue_test.default.http_logs size <= 0 AND like(request, 'GET%') | head 10",SUCCESS -"source = myglue_test.default.http_logs substring(clientip, 5, 2) = ""12"" | head 10",SUCCESS -source = myglue_test.default.http_logs | where isempty(size),FAILED -source = myglue_test.default.http_logs | where ispresent(size),FAILED -source = myglue_test.default.http_logs | where isnull(size) | head 10,SUCCESS -source = myglue_test.default.http_logs | where isnotnull(size) | head 10,SUCCESS -"source = myglue_test.default.http_logs | where isnotnull(coalesce(size, status)) | head 10",FAILED -"source = myglue_test.default.http_logs | where like(request, 'GET%') | head 10",SUCCESS -"source = myglue_test.default.http_logs | where like(request, '%bordeaux%') | head 10",SUCCESS -"source = myglue_test.default.http_logs | where substring(clientip, 5, 2) = ""12"" | head 10",SUCCESS -"source = myglue_test.default.http_logs | where lower(request) = ""get /images/backnews.gif http/1.0"" | head 10",SUCCESS -source = myglue_test.default.http_logs | where length(request) = 38 | head 10,SUCCESS -"source = myglue_test.default.http_logs | where case(status = 200, 'success' else 'failed') = 'success' | head 10",FAILED -"source = myglue_test.default.http_logs | eval h = ""Hello"", w = ""World"" | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval @h = ""Hello"" | eval @w = ""World"" | fields @timestamp, @h, @w",SUCCESS -source = myglue_test.default.http_logs | eval newF = clientip | head 10,SUCCESS -"source = myglue_test.default.http_logs | eval newF = clientip | fields clientip, newF | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval f = size | where f > 1 | sort f | fields size, clientip, status | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval f = status * 2 | eval h = f * 2 | fields status, f, h | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval f = size * 2, h = status | stats sum(f) by h",SUCCESS -"source = myglue_test.default.http_logs | eval f = UPPER(request) | eval h = 40 | fields f, h | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval request = ""test"" | fields request | head 10",FAILED -source = myglue_test.default.http_logs | eval size = abs(size) | where size < 500,FAILED -"source = myglue_test.default.http_logs | eval status_string = case(status = 200, 'success' else 'failed') | head 10",FAILED -"source = myglue_test.default.http_logs | eval n = now() | eval t = unix_timestamp(@timestamp) | fields n, t | head 10",SUCCESS -source = myglue_test.default.http_logs | eval e = isempty(size) | eval p = ispresent(size) | head 10,FAILED -"source = myglue_test.default.http_logs | eval c = coalesce(size, status) | head 10",FAILED -source = myglue_test.default.http_logs | eval c = coalesce(request) | head 10,FAILED -source = myglue_test.default.http_logs | eval col1 = ln(size) | eval col2 = unix_timestamp(@timestamp) | sort - col1 | head 10,SUCCESS -"source = myglue_test.default.http_logs | eval col1 = 1 | sort col1 | head 4 | eval col2 = 2 | sort - col2 | sort - size | head 2 | fields @timestamp, clientip, col2",SUCCESS -"source = myglue_test.default.mini_http_logs | eval stat = status | where stat > 300 | sort stat | fields @timestamp,clientip,status | head 5",SUCCESS -"source = myglue_test.default.http_logs | eval col1 = size, col2 = clientip | stats avg(col1) by col2",SUCCESS -source = myglue_test.default.http_logs | stats avg(size) by clientip,SUCCESS -"source = myglue_test.default.http_logs | eval new_request = upper(request) | eval compound_field = concat('Hello ', if(like(new_request, '%bordeaux%'), 'World', clientip)) | fields new_request, compound_field | head 10",SUCCESS -source = myglue_test.default.http_logs | stats avg(size),SUCCESS -source = myglue_test.default.nested | stats max(int_col) by struct_col.field2,SUCCESS -source = myglue_test.default.nested | stats distinct_count(int_col),SUCCESS -source = myglue_test.default.nested | stats stddev_samp(int_col),SUCCESS -source = myglue_test.default.nested | stats stddev_pop(int_col),SUCCESS -source = myglue_test.default.nested | stats percentile(int_col),SUCCESS -source = myglue_test.default.nested | stats percentile_approx(int_col),SUCCESS -source = myglue_test.default.mini_http_logs | stats stddev_samp(status),SUCCESS -"source = myglue_test.default.mini_http_logs | where stats > 200 | stats percentile_approx(status, 99)",SUCCESS -"source = myglue_test.default.nested | stats count(int_col) by span(struct_col.field2, 10) as a_span",SUCCESS -"source = myglue_test.default.nested | stats avg(int_col) by span(struct_col.field2, 10) as a_span, struct_col2.field2",SUCCESS -"source = myglue_test.default.http_logs | stats sum(size) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10",SUCCESS -"source = myglue_test.default.http_logs | stats distinct_count(clientip) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10",SUCCESS -"source = myglue_test.default.http_logs | stats avg(size) as avg_size by status, year | stats avg(avg_size) as avg_avg_size by year",SUCCESS -"source = myglue_test.default.http_logs | stats avg(size) as avg_size by status, year, month | stats avg(avg_size) as avg_avg_size by year, month | stats avg(avg_avg_size) as avg_avg_avg_size by year",SUCCESS -"source = myglue_test.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | stats avg(avg_int) as avg_avg_int by struct_col2.field2",FAILED -"source = myglue_test.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | eval new_col = avg_int | stats avg(avg_int) as avg_avg_int by new_col",SUCCESS -source = myglue_test.default.nested | rare int_col,SUCCESS -source = myglue_test.default.nested | rare int_col by struct_col.field2,SUCCESS -source = myglue_test.default.http_logs | rare request,SUCCESS -source = myglue_test.default.http_logs | where status > 300 | rare request by status,SUCCESS -source = myglue_test.default.http_logs | rare clientip,SUCCESS -source = myglue_test.default.http_logs | where status > 300 | rare clientip,SUCCESS -source = myglue_test.default.http_logs | where status > 300 | rare clientip by day,SUCCESS -source = myglue_test.default.nested | top int_col by struct_col.field2,SUCCESS -source = myglue_test.default.nested | top 1 int_col by struct_col.field2,SUCCESS -source = myglue_test.default.nested | top 2 int_col by struct_col.field2,SUCCESS -source = myglue_test.default.nested | top int_col,SUCCESS -source = myglue_test.default.http_logs | inner join left=l right=r on l.status = r.int_col myglue_test.default.nested | head 10,FAILED -"source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | fields request, domain | head 10",SUCCESS -source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | top 1 domain,SUCCESS -source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | stats count() by domain,SUCCESS -"source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | eval a = 1 | fields a, domain | head 10",SUCCESS -"source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | where size > 0 | sort - size | fields size, domain | head 10",SUCCESS -"source = myglue_test.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/(?[a-zA-Z]+)/.*' | where domain = 'english' | sort - picName | fields domain, picName | head 10",SUCCESS -source = myglue_test.default.http_logs | patterns request | fields patterns_field | head 10,SUCCESS -source = myglue_test.default.http_logs | patterns request | where size > 0 | fields patterns_field | head 10,SUCCESS -"source = myglue_test.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | fields request, no_letter | head 10",SUCCESS -source = myglue_test.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | stats count() by no_letter,SUCCESS -"source = myglue_test.default.http_logs | patterns new_field='status' pattern='[a-zA-Z]' request | fields request, status | head 10",FAILED -source = myglue_test.default.http_logs | rename @timestamp as timestamp | head 10,FAILED -source = myglue_test.default.http_logs | sort size | head 10,SUCCESS -source = myglue_test.default.http_logs | sort + size | head 10,SUCCESS -source = myglue_test.default.http_logs | sort - size | head 10,SUCCESS -"source = myglue_test.default.http_logs | sort + size, + @timestamp | head 10",SUCCESS -"source = myglue_test.default.http_logs | sort - size, - @timestamp | head 10",SUCCESS -"source = myglue_test.default.http_logs | sort - size, @timestamp | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval c1 = upper(request) | eval c2 = concat('Hello ', if(like(c1, '%bordeaux%'), 'World', clientip)) | eval c3 = length(request) | eval c4 = ltrim(request) | eval c5 = rtrim(request) | eval c6 = substring(clientip, 5, 2) | eval c7 = trim(request) | eval c8 = upper(request) | eval c9 = position('bordeaux' IN request) | eval c10 = replace(request, 'GET', 'GGG') | fields c1, c2, c3, c4, c5, c6, c7, c8, c9, c10 | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval c1 = unix_timestamp(@timestamp) | eval c2 = now() | eval c3 = +describe dev.default.http_logs,FAILED +describe `dev`.`default`.`http_logs`,FAILED +"source = dev.default.http_logs | dedup 1 status | fields @timestamp, clientip, status, size | head 10",SUCCESS +"source = dev.default.http_logs | dedup status, size | head 10",SUCCESS +source = dev.default.http_logs | dedup 1 status keepempty=true | head 10,SUCCESS +"source = dev.default.http_logs | dedup status, size keepempty=true | head 10",SUCCESS +source = dev.default.http_logs | dedup 2 status | head 10,SUCCESS +"source = dev.default.http_logs | dedup 2 status, size | head 10",SUCCESS +"source = dev.default.http_logs | dedup 2 status, size keepempty=true | head 10",SUCCESS +source = dev.default.http_logs | dedup status CONSECUTIVE=true | fields status,FAILED +"source = dev.default.http_logs | dedup 2 status, size CONSECUTIVE=true | fields status",FAILED +"source = dev.default.http_logs | sort status | fields @timestamp, clientip, status | head 10",SUCCESS +"source = dev.default.http_logs | fields @timestamp, notexisted | head 10",FAILED +"source = dev.default.nested | fields int_col, struct_col.field1, struct_col2.field1 | head 10",SUCCESS +"source = dev.default.nested | where struct_col2.field1.subfield > 'valueA' | sort int_col | fields int_col, struct_col.field1.subfield, struct_col2.field1.subfield",SUCCESS +"source = dev.default.http_logs | fields - @timestamp, clientip, status | head 10",SUCCESS +"source = dev.default.http_logs | eval new_time = @timestamp, new_clientip = clientip | fields - new_time, new_clientip, status | head 10",SUCCESS +source = dev.default.http_logs | eval new_clientip = lower(clientip) | fields - new_clientip | head 10,SUCCESS +"source = dev.default.http_logs | fields + @timestamp, clientip, status | fields - clientip, status | head 10",SUCCESS +"source = dev.default.http_logs | fields - clientip, status | fields + @timestamp, clientip, status| head 10",SUCCESS +source = dev.default.http_logs | where status = 200 | head 10,SUCCESS +source = dev.default.http_logs | where status != 200 | head 10,SUCCESS +source = dev.default.http_logs | where size > 0 | head 10,SUCCESS +source = dev.default.http_logs | where size <= 0 | head 10,SUCCESS +source = dev.default.http_logs | where clientip = '236.14.2.0' | head 10,SUCCESS +source = dev.default.http_logs | where size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100,SUCCESS +"source = dev.default.http_logs | where size <= 0 AND like(request, 'GET%') | head 10",SUCCESS +source = dev.default.http_logs status = 200 | head 10,SUCCESS +source = dev.default.http_logs size > 0 AND status = 200 OR clientip = '236.14.2.0' | head 100,SUCCESS +"source = dev.default.http_logs size <= 0 AND like(request, 'GET%') | head 10",SUCCESS +"source = dev.default.http_logs substring(clientip, 5, 2) = ""12"" | head 10",SUCCESS +source = dev.default.http_logs | where isempty(size),SUCCESS +source = dev.default.http_logs | where ispresent(size),SUCCESS +source = dev.default.http_logs | where isnull(size) | head 10,SUCCESS +source = dev.default.http_logs | where isnotnull(size) | head 10,SUCCESS +"source = dev.default.http_logs | where isnotnull(coalesce(size, status)) | head 10",SUCCESS +"source = dev.default.http_logs | where like(request, 'GET%') | head 10",SUCCESS +"source = dev.default.http_logs | where like(request, '%bordeaux%') | head 10",SUCCESS +"source = dev.default.http_logs | where substring(clientip, 5, 2) = ""12"" | head 10",SUCCESS +"source = dev.default.http_logs | where lower(request) = ""get /images/backnews.gif http/1.0"" | head 10",SUCCESS +source = dev.default.http_logs | where length(request) = 38 | head 10,SUCCESS +"source = dev.default.http_logs | where case(status = 200, 'success' else 'failed') = 'success' | head 10",SUCCESS +"source = dev.default.http_logs | eval h = ""Hello"", w = ""World"" | head 10",SUCCESS +"source = dev.default.http_logs | eval @h = ""Hello"" | eval @w = ""World"" | fields @timestamp, @h, @w",SUCCESS +source = dev.default.http_logs | eval newF = clientip | head 10,SUCCESS +"source = dev.default.http_logs | eval newF = clientip | fields clientip, newF | head 10",SUCCESS +"source = dev.default.http_logs | eval f = size | where f > 1 | sort f | fields size, clientip, status | head 10",SUCCESS +"source = dev.default.http_logs | eval f = status * 2 | eval h = f * 2 | fields status, f, h | head 10",SUCCESS +"source = dev.default.http_logs | eval f = size * 2, h = status | stats sum(f) by h",SUCCESS +"source = dev.default.http_logs | eval f = UPPER(request) | eval h = 40 | fields f, h | head 10",SUCCESS +"source = dev.default.http_logs | eval request = ""test"" | fields request | head 10",FAILED +source = dev.default.http_logs | eval size = abs(size) | where size < 500,FAILED +"source = dev.default.http_logs | eval status_string = case(status = 200, 'success' else 'failed') | head 10",SUCCESS +"source = dev.default.http_logs | eval n = now() | eval t = unix_timestamp(@timestamp) | fields n, t | head 10",SUCCESS +source = dev.default.http_logs | eval e = isempty(size) | eval p = ispresent(size) | head 10,SUCCESS +"source = dev.default.http_logs | eval c = coalesce(size, status) | head 10",SUCCESS +source = dev.default.http_logs | eval c = coalesce(request) | head 10,SUCCESS +source = dev.default.http_logs | eval col1 = ln(size) | eval col2 = unix_timestamp(@timestamp) | sort - col1 | head 10,SUCCESS +"source = dev.default.http_logs | eval col1 = 1 | sort col1 | head 4 | eval col2 = 2 | sort - col2 | sort - size | head 2 | fields @timestamp, clientip, col2",SUCCESS +"source = dev.default.http_logs | eval stat = status | where stat > 300 | sort stat | fields @timestamp,clientip,status | head 5",SUCCESS +"source = dev.default.http_logs | eval col1 = size, col2 = clientip | stats avg(col1) by col2",SUCCESS +source = dev.default.http_logs | stats avg(size) by clientip,SUCCESS +"source = dev.default.http_logs | eval new_request = upper(request) | eval compound_field = concat('Hello ', if(like(new_request, '%bordeaux%'), 'World', clientip)) | fields new_request, compound_field | head 10",SUCCESS +source = dev.default.http_logs | stats avg(size),SUCCESS +source = dev.default.nested | stats max(int_col) by struct_col.field2,SUCCESS +source = dev.default.nested | stats distinct_count(int_col),SUCCESS +source = dev.default.nested | stats stddev_samp(int_col),SUCCESS +source = dev.default.nested | stats stddev_pop(int_col),SUCCESS +"source = dev.default.nested | stats percentile(int_col, 90)",SUCCESS +"source = dev.default.nested | stats percentile_approx(int_col, 99)",SUCCESS +source = dev.default.http_logs | stats stddev_samp(status),SUCCESS +"source = dev.default.http_logs | where status > 200 | stats percentile_approx(status, 99)",SUCCESS +"source = dev.default.nested | stats count(int_col) by span(struct_col.field2, 10) as a_span",SUCCESS +"source = dev.default.nested | stats avg(int_col) by span(struct_col.field2, 10) as a_span, struct_col2.field2",SUCCESS +"source = dev.default.http_logs | stats sum(size) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10",SUCCESS +"source = dev.default.http_logs | stats distinct_count(clientip) by span(@timestamp, 1d) as age_size_per_day | sort - age_size_per_day | head 10",SUCCESS +"source = dev.default.http_logs | stats avg(size) as avg_size by status, year | stats avg(avg_size) as avg_avg_size by year",SUCCESS +"source = dev.default.http_logs | stats avg(size) as avg_size by status, year, month | stats avg(avg_size) as avg_avg_size by year, month | stats avg(avg_avg_size) as avg_avg_avg_size by year",SUCCESS +"source = dev.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | stats avg(avg_int) as avg_avg_int by struct_col2.field2",FAILED +"source = dev.default.nested | stats avg(int_col) as avg_int by struct_col.field2, struct_col2.field2 | eval new_col = avg_int | stats avg(avg_int) as avg_avg_int by new_col",SUCCESS +source = dev.default.nested | rare int_col,SUCCESS +source = dev.default.nested | rare int_col by struct_col.field2,SUCCESS +source = dev.default.http_logs | rare request,SUCCESS +source = dev.default.http_logs | where status > 300 | rare request by status,SUCCESS +source = dev.default.http_logs | rare clientip,SUCCESS +source = dev.default.http_logs | where status > 300 | rare clientip,SUCCESS +source = dev.default.http_logs | where status > 300 | rare clientip by day,SUCCESS +source = dev.default.nested | top int_col by struct_col.field2,SUCCESS +source = dev.default.nested | top 1 int_col by struct_col.field2,SUCCESS +source = dev.default.nested | top 2 int_col by struct_col.field2,SUCCESS +source = dev.default.nested | top int_col,SUCCESS +source = dev.default.http_logs | inner join left=l right=r on l.status = r.int_col dev.default.nested | head 10,SUCCESS +"source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | fields request, domain | head 10",SUCCESS +source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | top 1 domain,SUCCESS +source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | stats count() by domain,SUCCESS +"source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | eval a = 1 | fields a, domain | head 10",SUCCESS +"source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/.*' | where size > 0 | sort - size | fields size, domain | head 10",SUCCESS +"source = dev.default.http_logs | parse request 'GET /(?[a-zA-Z]+)/(?[a-zA-Z]+)/.*' | where domain = 'english' | sort - picName | fields domain, picName | head 10",SUCCESS +source = dev.default.http_logs | patterns request | fields patterns_field | head 10,SUCCESS +source = dev.default.http_logs | patterns request | where size > 0 | fields patterns_field | head 10,SUCCESS +"source = dev.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | fields request, no_letter | head 10",SUCCESS +source = dev.default.http_logs | patterns new_field='no_letter' pattern='[a-zA-Z]' request | stats count() by no_letter,SUCCESS +"source = dev.default.http_logs | patterns new_field='status' pattern='[a-zA-Z]' request | fields request, status | head 10",FAILED +source = dev.default.http_logs | rename @timestamp as timestamp | head 10,SUCCESS +source = dev.default.http_logs | sort size | head 10,SUCCESS +source = dev.default.http_logs | sort + size | head 10,SUCCESS +source = dev.default.http_logs | sort - size | head 10,SUCCESS +"source = dev.default.http_logs | sort + size, + @timestamp | head 10",SUCCESS +"source = dev.default.http_logs | sort - size, - @timestamp | head 10",SUCCESS +"source = dev.default.http_logs | sort - size, @timestamp | head 10",SUCCESS +"source = dev.default.http_logs | eval c1 = upper(request) | eval c2 = concat('Hello ', if(like(c1, '%bordeaux%'), 'World', clientip)) | eval c3 = length(request) | eval c4 = ltrim(request) | eval c5 = rtrim(request) | eval c6 = substring(clientip, 5, 2) | eval c7 = trim(request) | eval c8 = upper(request) | eval c9 = position('bordeaux' IN request) | eval c10 = replace(request, 'GET', 'GGG') | fields c1, c2, c3, c4, c5, c6, c7, c8, c9, c10 | head 10",SUCCESS +"source = dev.default.http_logs | eval c1 = unix_timestamp(@timestamp) | eval c2 = now() | eval c3 = DAY_OF_WEEK(@timestamp) | eval c4 = DAY_OF_MONTH(@timestamp) | eval c5 = DAY_OF_YEAR(@timestamp) | eval c6 = @@ -121,151 +121,151 @@ HOUR_OF_DAY(@timestamp) | eval c10 = MINUTE_OF_HOUR(@timestamp) | eval c11 = SECOND_OF_MINUTE(@timestamp) | eval c12 = LOCALTIME() | fields c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12 | head 10",SUCCESS -"source=myglue_test.default.people | eval c1 = adddate(@timestamp, 1) | fields c1 | head 10",SUCCESS -"source=myglue_test.default.people | eval c2 = subdate(@timestamp, 1) | fields c2 | head 10",SUCCESS -source=myglue_test.default.people | eval c1 = date_add(@timestamp INTERVAL 1 DAY) | fields c1 | head 10,SUCCESS -source=myglue_test.default.people | eval c1 = date_sub(@timestamp INTERVAL 1 DAY) | fields c1 | head 10,SUCCESS -source=myglue_test.default.people | eval `CURDATE()` = CURDATE() | fields `CURDATE()`,SUCCESS -source=myglue_test.default.people | eval `CURRENT_DATE()` = CURRENT_DATE() | fields `CURRENT_DATE()`,SUCCESS -source=myglue_test.default.people | eval `CURRENT_TIMESTAMP()` = CURRENT_TIMESTAMP() | fields `CURRENT_TIMESTAMP()`,SUCCESS -source=myglue_test.default.people | eval `DATE('2020-08-26')` = DATE('2020-08-26') | fields `DATE('2020-08-26')`,SUCCESS -source=myglue_test.default.people | eval `DATE(TIMESTAMP('2020-08-26 13:49:00'))` = DATE(TIMESTAMP('2020-08-26 13:49:00')) | fields `DATE(TIMESTAMP('2020-08-26 13:49:00'))`,SUCCESS -source=myglue_test.default.people | eval `DATE('2020-08-26 13:49')` = DATE('2020-08-26 13:49') | fields `DATE('2020-08-26 13:49')`,SUCCESS -"source=myglue_test.default.people | eval `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')` = DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS'), `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')` = DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a') | fields `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')`, `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')`",SUCCESS -"source=myglue_test.default.people | eval `'2000-01-02' - '2000-01-01'` = DATEDIFF(TIMESTAMP('2000-01-02 00:00:00'), TIMESTAMP('2000-01-01 23:59:59')), `'2001-02-01' - '2004-01-01'` = DATEDIFF(DATE('2001-02-01'), TIMESTAMP('2004-01-01 00:00:00')) | fields `'2000-01-02' - '2000-01-01'`, `'2001-02-01' - '2004-01-01'`", -source=myglue_test.default.people | eval `DAY(DATE('2020-08-26'))` = DAY(DATE('2020-08-26')) | fields `DAY(DATE('2020-08-26'))`, -source=myglue_test.default.people | eval `DAYNAME(DATE('2020-08-26'))` = DAYNAME(DATE('2020-08-26')) | fields `DAYNAME(DATE('2020-08-26'))`,FAILED -source=myglue_test.default.people | eval `CURRENT_TIMEZONE()` = CURRENT_TIMEZONE() | fields `CURRENT_TIMEZONE()`,SUCCESS -source=myglue_test.default.people | eval `UTC_TIMESTAMP()` = UTC_TIMESTAMP() | fields `UTC_TIMESTAMP()`,SUCCESS -"source=myglue_test.default.people | eval `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')` = TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00') | eval `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))` = TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00')) | fields `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')`, `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))`",SUCCESS -"source=myglue_test.default.people | eval `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')` = TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00') | eval `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')` = TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00') | fields `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')`, `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')`",SUCCESS - source = myglue_test.default.http_logs | stats count(),SUCCESS -"source = myglue_test.default.http_logs | stats avg(size) as c1, max(size) as c2, min(size) as c3, sum(size) as c4, percentile(size, 50) as c5, stddev_pop(size) as c6, stddev_samp(size) as c7, distinct_count(size) as c8",SUCCESS -"source = myglue_test.default.http_logs | eval c1 = abs(size) | eval c2 = ceil(size) | eval c3 = floor(size) | eval c4 = sqrt(size) | eval c5 = ln(size) | eval c6 = pow(size, 2) | eval c7 = mod(size, 2) | fields c1, c2, c3, c4, c5, c6, c7 | head 10",SUCCESS -"source = myglue_test.default.http_logs | eval c1 = isnull(request) | eval c2 = isnotnull(request) | eval c3 = ifnull(request, +"source=dev.default.people | eval c1 = adddate(@timestamp, 1) | fields c1 | head 10",SUCCESS +"source=dev.default.people | eval c2 = subdate(@timestamp, 1) | fields c2 | head 10",SUCCESS +source=dev.default.people | eval c1 = date_add(@timestamp INTERVAL 1 DAY) | fields c1 | head 10,SUCCESS +source=dev.default.people | eval c1 = date_sub(@timestamp INTERVAL 1 DAY) | fields c1 | head 10,SUCCESS +source=dev.default.people | eval `CURDATE()` = CURDATE() | fields `CURDATE()`,SUCCESS +source=dev.default.people | eval `CURRENT_DATE()` = CURRENT_DATE() | fields `CURRENT_DATE()`,SUCCESS +source=dev.default.people | eval `CURRENT_TIMESTAMP()` = CURRENT_TIMESTAMP() | fields `CURRENT_TIMESTAMP()`,SUCCESS +source=dev.default.people | eval `DATE('2020-08-26')` = DATE('2020-08-26') | fields `DATE('2020-08-26')`,SUCCESS +source=dev.default.people | eval `DATE(TIMESTAMP('2020-08-26 13:49:00'))` = DATE(TIMESTAMP('2020-08-26 13:49:00')) | fields `DATE(TIMESTAMP('2020-08-26 13:49:00'))`,SUCCESS +source=dev.default.people | eval `DATE('2020-08-26 13:49')` = DATE('2020-08-26 13:49') | fields `DATE('2020-08-26 13:49')`,SUCCESS +"source=dev.default.people | eval `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')` = DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS'), `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')` = DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a') | fields `DATE_FORMAT('1998-01-31 13:14:15.012345', 'HH:mm:ss.SSSSSS')`, `DATE_FORMAT(TIMESTAMP('1998-01-31 13:14:15.012345'), 'yyyy-MMM-dd hh:mm:ss a')`",SUCCESS +"source=dev.default.people | eval `'2000-01-02' - '2000-01-01'` = DATEDIFF(TIMESTAMP('2000-01-02 00:00:00'), TIMESTAMP('2000-01-01 23:59:59')), `'2001-02-01' - '2004-01-01'` = DATEDIFF(DATE('2001-02-01'), TIMESTAMP('2004-01-01 00:00:00')) | fields `'2000-01-02' - '2000-01-01'`, `'2001-02-01' - '2004-01-01'`",SUCCESS +source=dev.default.people | eval `DAY(DATE('2020-08-26'))` = DAY(DATE('2020-08-26')) | fields `DAY(DATE('2020-08-26'))`,SUCCESS +source=dev.default.people | eval `DAYNAME(DATE('2020-08-26'))` = DAYNAME(DATE('2020-08-26')) | fields `DAYNAME(DATE('2020-08-26'))`,FAILED +source=dev.default.people | eval `CURRENT_TIMEZONE()` = CURRENT_TIMEZONE() | fields `CURRENT_TIMEZONE()`,SUCCESS +source=dev.default.people | eval `UTC_TIMESTAMP()` = UTC_TIMESTAMP() | fields `UTC_TIMESTAMP()`,SUCCESS +"source=dev.default.people | eval `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')` = TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00') | eval `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))` = TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00')) | fields `TIMESTAMPDIFF(YEAR, '1997-01-01 00:00:00', '2001-03-06 00:00:00')`, `TIMESTAMPDIFF(SECOND, timestamp('1997-01-01 00:00:23'), timestamp('1997-01-01 00:00:00'))`",SUCCESS +"source=dev.default.people | eval `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')` = TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00') | eval `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')` = TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00') | fields `TIMESTAMPADD(DAY, 17, '2000-01-01 00:00:00')`, `TIMESTAMPADD(QUARTER, -1, '2000-01-01 00:00:00')`",SUCCESS + source = dev.default.http_logs | stats count(),SUCCESS +"source = dev.default.http_logs | stats avg(size) as c1, max(size) as c2, min(size) as c3, sum(size) as c4, percentile(size, 50) as c5, stddev_pop(size) as c6, stddev_samp(size) as c7, distinct_count(size) as c8",SUCCESS +"source = dev.default.http_logs | eval c1 = abs(size) | eval c2 = ceil(size) | eval c3 = floor(size) | eval c4 = sqrt(size) | eval c5 = ln(size) | eval c6 = pow(size, 2) | eval c7 = mod(size, 2) | fields c1, c2, c3, c4, c5, c6, c7 | head 10",SUCCESS +"source = dev.default.http_logs | eval c1 = isnull(request) | eval c2 = isnotnull(request) | eval c3 = ifnull(request, ""Unknown"") | eval c4 = nullif(request, ""Unknown"") | eval c5 = isnull(size) | eval c6 = if(like(request, '%bordeaux%'), 'hello', 'world') | fields c1, c2, c3, c4, c5, c6 | head 10",SUCCESS -/* this is block comment */ source = myglue_test.tpch_csv.orders | head 1 // this is line comment,SUCCESS -"/* test in tpch q16, q18, q20 */ source = myglue_test.tpch_csv.orders | head 1 // add source=xx to avoid failure in automation",SUCCESS -"/* test in tpch q4, q21, q22 */ source = myglue_test.tpch_csv.orders | head 1",SUCCESS -"/* test in tpch q2, q11, q15, q17, q20, q22 */ source = myglue_test.tpch_csv.orders | head 1",SUCCESS -"/* test in tpch q7, q8, q9, q13, q15, q22 */ source = myglue_test.tpch_csv.orders | head 1",SUCCESS -/* lots of inner join tests in tpch */ source = myglue_test.tpch_csv.orders | head 1,SUCCESS -/* left join test in tpch q13 */ source = myglue_test.tpch_csv.orders | head 1,SUCCESS -"source = myglue_test.tpch_csv.orders +/* this is block comment */ source = dev.default.orders | head 1 // this is line comment,SUCCESS +"/* test in tpch q16, q18, q20 */ source = dev.default.orders | head 1 // add source=xx to avoid failure in automation",SUCCESS +"/* test in tpch q4, q21, q22 */ source = dev.default.orders | head 1",SUCCESS +"/* test in tpch q2, q11, q15, q17, q20, q22 */ source = dev.default.orders | head 1",SUCCESS +"/* test in tpch q7, q8, q9, q13, q15, q22 */ source = dev.default.orders | head 1",SUCCESS +/* lots of inner join tests in tpch */ source = dev.default.orders | head 1,SUCCESS +/* left join test in tpch q13 */ source = dev.default.orders | head 1,SUCCESS +"source = dev.default.orders | right outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') - myglue_test.tpch_csv.customer + dev.default.customer | stats count(o_orderkey) as c_count by c_custkey | sort - c_count",SUCCESS -"source = myglue_test.tpch_csv.orders +"source = dev.default.orders | full outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') - myglue_test.tpch_csv.customer + dev.default.customer | stats count(o_orderkey) as c_count by c_custkey | sort - c_count",SUCCESS -"source = myglue_test.tpch_csv.customer -| semi join ON c_custkey = o_custkey myglue_test.tpch_csv.orders +"source = dev.default.customer +| semi join ON c_custkey = o_custkey dev.default.orders | where c_mktsegment = 'BUILDING' | sort - c_custkey | head 10",SUCCESS -"source = myglue_test.tpch_csv.customer -| anti join ON c_custkey = o_custkey myglue_test.tpch_csv.orders +"source = dev.default.customer +| anti join ON c_custkey = o_custkey dev.default.orders | where c_mktsegment = 'BUILDING' | sort - c_custkey | head 10",SUCCESS -"source = myglue_test.tpch_csv.supplier +"source = dev.default.supplier | where like(s_comment, '%Customer%Complaints%') -| join ON s_nationkey > n_nationkey [ source = myglue_test.tpch_csv.nation | where n_name = 'SAUDI ARABIA' ] +| join ON s_nationkey > n_nationkey [ source = dev.default.nation | where n_name = 'SAUDI ARABIA' ] | sort - s_name | head 10",SUCCESS -"source = myglue_test.tpch_csv.supplier +"source = dev.default.supplier | where like(s_comment, '%Customer%Complaints%') -| join [ source = myglue_test.tpch_csv.nation | where n_name = 'SAUDI ARABIA' ] +| join [ source = dev.default.nation | where n_name = 'SAUDI ARABIA' ] | sort - s_name | head 10",SUCCESS -source=myglue_test.default.people | LOOKUP myglue_test.default.work_info uid AS id REPLACE department | stats distinct_count(department),SUCCESS -source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id APPEND department | stats distinct_count(department),SUCCESS -source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id REPLACE department AS country | stats distinct_count(country),SUCCESS -source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS id APPEND department AS country | stats distinct_count(country),SUCCESS -"source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uID AS id, name REPLACE department | stats distinct_count(department)",SUCCESS -"source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uid AS ID, name APPEND department | stats distinct_count(department)",SUCCESS -"source = myglue_test.default.people| LOOKUP myglue_test.default.work_info uID AS id, name | head 10",SUCCESS -"source = myglue_test.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP myglue_test.default.work_info name REPLACE occupation AS major | stats distinct_count(major)",SUCCESS -"source = myglue_test.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP myglue_test.default.work_info name APPEND occupation AS major | stats distinct_count(major)",SUCCESS -"source = myglue_test.default.http_logs | eval res = json('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json('{""f1"":""abc"",""f2"":{""f3"":""a"",""f4"":""b""}}') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json('[1,2,3,{""f1"":1,""f2"":[5,6]},4]') | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = json('[]') | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json(‘{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json('{""invalid"": ""json""') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json('[1,2,3]') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json(‘[1,2') | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = json('[invalid json]') | head 1 | fields res,SUCCESS -source = myglue_test.default.http_logs | eval res = json('invalid json') | head 1 | fields res,SUCCESS -source = myglue_test.default.http_logs | eval res = json(null) | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array('this', 'is', 'a', 'string', 'array') | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = json_array() | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array(1, 2, 0, -1, 1.1, -0.11) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array('this', 'is', 1.1, -0.11, true, false) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = array_length(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = array_length(json_array()) | head 1 | fields res,SUCCESS -source = myglue_test.default.http_logs | eval res = json_array_length('[]') | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array_length('[1,2,3,{""f1"":1,""f2"":[5,6]},4]') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array_length('{\""key\"": 1}') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_array_length('[1,2') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', 'string_value')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', 123.45)) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', true)) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object(""a"", 1, ""b"", 2, ""c"", 3)) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', array())) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('key', array(1, 2, 3))) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object('outer', json_object('inner', 123.45))) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = to_json_string(json_object(""array"", json_array(1,2,0,-1,1.1,-0.11))) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | where json_valid(('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1",SUCCESS -"source = myglue_test.default.http_logs | where not json_valid(('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('{""f1"":""abc"",""f2"":{""f3"":""a"",""f4"":""b""}}')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2,3,{""f1"":1,""f2"":[5,6]},4]')) | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = json_keys(json('[]')) | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json(‘{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('{""invalid"": ""json""')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2,3]')) | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_keys(json('[1,2')) | head 1 | fields res",SUCCESS -source = myglue_test.default.http_logs | eval res = json_keys(json('[invalid json]')) | head 1 | fields res,SUCCESS -source = myglue_test.default.http_logs | eval res = json_keys(json('invalid json')) | head 1 | fields res,SUCCESS -source = myglue_test.default.http_logs | eval res = json_keys(json(null)) | head 1 | fields res,SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.teacher') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[*]') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[0]') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[*].name') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[1].name') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[0].not_exist_key') | head 1 | fields res",SUCCESS -"source = myglue_test.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[10]') | head 1 | fields res",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > 0) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > -10) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(json_object(""a"",1,""b"",-1),json_object(""a"",-1,""b"",-1)), result = forall(array, x -> x.a > 0) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(json_object(""a"",1,""b"",-1),json_object(""a"",-1,""b"",-1)), result = exists(array, x -> x.b < 0) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 0) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 10) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 0) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 10) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,3), result = transform(array, x -> x + 1) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,3), result = transform(array, (x, y) -> x + y) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x) | head 1 | fields result",SUCCESS -"source = myglue_test.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x, acc -> acc * 10) | head 1 | fields result",SUCCESS -source=myglue_test.default.people | eval age = salary | eventstats avg(age) | sort id | head 10,SUCCESS -"source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count | sort id | head 10",SUCCESS -source=myglue_test.default.people | eventstats avg(salary) by country | sort id | head 10,SUCCESS -"source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by country | sort id | head 10",SUCCESS -"source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count +source=dev.default.people | LOOKUP dev.default.work_info uid AS id REPLACE department | stats distinct_count(department),SUCCESS +source = dev.default.people| LOOKUP dev.default.work_info uid AS id APPEND department | stats distinct_count(department),SUCCESS +source = dev.default.people| LOOKUP dev.default.work_info uid AS id REPLACE department AS country | stats distinct_count(country),SUCCESS +source = dev.default.people| LOOKUP dev.default.work_info uid AS id APPEND department AS country | stats distinct_count(country),SUCCESS +"source = dev.default.people| LOOKUP dev.default.work_info uID AS id, name REPLACE department | stats distinct_count(department)",SUCCESS +"source = dev.default.people| LOOKUP dev.default.work_info uid AS ID, name APPEND department | stats distinct_count(department)",SUCCESS +"source = dev.default.people| LOOKUP dev.default.work_info uID AS id, name | head 10",SUCCESS +"source = dev.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP dev.default.work_info name REPLACE occupation AS major | stats distinct_count(major)",SUCCESS +"source = dev.default.people | eval major = occupation | fields id, name, major, country, salary | LOOKUP dev.default.work_info name APPEND occupation AS major | stats distinct_count(major)",SUCCESS +"source = dev.default.http_logs | eval res = json('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json('{""f1"":""abc"",""f2"":{""f3"":""a"",""f4"":""b""}}') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json('[1,2,3,{""f1"":1,""f2"":[5,6]},4]') | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = json('[]') | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json('{""invalid"": ""json""') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json('[1,2,3]') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json('[1,2') | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = json('[invalid json]') | head 1 | fields res,SUCCESS +source = dev.default.http_logs | eval res = json('invalid json') | head 1 | fields res,SUCCESS +source = dev.default.http_logs | eval res = json(null) | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json_array('this', 'is', 'a', 'string', 'array') | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = json_array() | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json_array(1, 2, 0, -1, 1.1, -0.11) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_array('this', 'is', 1.1, -0.11, true, false) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = array_length(json_array(1,2,0,-1,1.1,-0.11)) | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = array_length(json_array()) | head 1 | fields res,SUCCESS +source = dev.default.http_logs | eval res = json_array_length('[]') | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json_array_length('[1,2,3,{""f1"":1,""f2"":[5,6]},4]') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_array_length('{\""key\"": 1}') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_array_length('[1,2') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('key', 'string_value')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('key', 123.45)) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('key', true)) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object(""a"", 1, ""b"", 2, ""c"", 3)) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('key', array())) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('key', array(1, 2, 3))) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object('outer', json_object('inner', 123.45))) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = to_json_string(json_object(""array"", json_array(1,2,0,-1,1.1,-0.11))) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | where json_valid('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1",SUCCESS +"source = dev.default.http_logs | where not json_valid('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}') | head 1",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('{""account_number"":1,""balance"":39225,""age"":32,""gender"":""M""}')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('{""f1"":""abc"",""f2"":{""f3"":""a"",""f4"":""b""}}')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('[1,2,3,{""f1"":1,""f2"":[5,6]},4]')) | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = json_keys(json('[]')) | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('{""invalid"": ""json""')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('[1,2,3]')) | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_keys(json('[1,2')) | head 1 | fields res",SUCCESS +source = dev.default.http_logs | eval res = json_keys(json('[invalid json]')) | head 1 | fields res,SUCCESS +source = dev.default.http_logs | eval res = json_keys(json('invalid json')) | head 1 | fields res,SUCCESS +source = dev.default.http_logs | eval res = json_keys(json(null)) | head 1 | fields res,SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.teacher') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[*]') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[0]') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[*].name') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[1].name') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[0].not_exist_key') | head 1 | fields res",SUCCESS +"source = dev.default.http_logs | eval res = json_extract('{""teacher"":""Alice"",""student"":[{""name"":""Bob"",""rank"":1},{""name"":""Charlie"",""rank"":2}]}', '$.student[10]') | head 1 | fields res",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > 0) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = forall(array, x -> x > -10) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(json_object(""a"",1,""b"",-1),json_object(""a"",-1,""b"",-1)), result = forall(array, x -> x.a > 0) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(json_object(""a"",1,""b"",-1),json_object(""a"",-1,""b"",-1)), result = exists(array, x -> x.b < 0) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 0) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = exists(array, x -> x > 10) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 0) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,0,-1,1.1,-0.11), result = filter(array, x -> x > 10) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,3), result = transform(array, x -> x + 1) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,3), result = transform(array, (x, y) -> x + y) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x) | head 1 | fields result",SUCCESS +"source = dev.default.people | eval array = json_array(1,2,3), result = reduce(array, 0, (acc, x) -> acc + x, acc -> acc * 10) | head 1 | fields result",SUCCESS +source=dev.default.people | eval age = salary | eventstats avg(age) | sort id | head 10,SUCCESS +"source=dev.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count | sort id | head 10",SUCCESS +source=dev.default.people | eventstats avg(salary) by country | sort id | head 10,SUCCESS +"source=dev.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by country | sort id | head 10",SUCCESS +"source=dev.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by span(age, 10) | sort id | head 10",SUCCESS -"source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by span(age, 10) as age_span, country | sort id | head 10",SUCCESS -"source=myglue_test.default.people | where country != 'USA' | eventstats stddev_samp(salary), stddev_pop(salary), percentile_approx(salary, 60) by span(salary, 1000) as salary_span | sort id | head 10",SUCCESS -"source=myglue_test.default.people | eval age = salary | eventstats avg(age) as avg_age by occupation, country | eventstats avg(avg_age) as avg_state_age by country | sort id | head 10",SUCCESS -"source=myglue_test.default.people | eventstats distinct_count(salary) by span(salary, 1000) as age_span",FAILED -"source = myglue_test.tpch_csv.lineitem +"source=dev.default.people | eval age = salary | eventstats avg(age) as avg_age, max(age) as max_age, min(age) as min_age, count(age) as count by span(age, 10) as age_span, country | sort id | head 10",SUCCESS +"source=dev.default.people | where country != 'USA' | eventstats stddev_samp(salary), stddev_pop(salary), percentile_approx(salary, 60) by span(salary, 1000) as salary_span | sort id | head 10",SUCCESS +"source=dev.default.people | eval age = salary | eventstats avg(age) as avg_age by occupation, country | eventstats avg(avg_age) as avg_state_age by country | sort id | head 10",SUCCESS +"source=dev.default.people | eventstats distinct_count(salary) by span(salary, 1000) as age_span",FAILED +"source = dev.default.lineitem | where l_shipdate <= subdate(date('1998-12-01'), 90) | stats sum(l_quantity) as sum_qty, sum(l_extendedprice) as sum_base_price, @@ -277,59 +277,59 @@ by span(age, 10) | sort id | head 10",SUCCESS count() as count_order by l_returnflag, l_linestatus | sort l_returnflag, l_linestatus",SUCCESS -"source = myglue_test.tpch_csv.part -| join ON p_partkey = ps_partkey myglue_test.tpch_csv.partsupp -| join ON s_suppkey = ps_suppkey myglue_test.tpch_csv.supplier -| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation -| join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region +"source = dev.default.part +| join ON p_partkey = ps_partkey dev.default.partsupp +| join ON s_suppkey = ps_suppkey dev.default.supplier +| join ON s_nationkey = n_nationkey dev.default.nation +| join ON n_regionkey = r_regionkey dev.default.region | where p_size = 15 AND like(p_type, '%BRASS') AND r_name = 'EUROPE' AND ps_supplycost = [ - source = myglue_test.tpch_csv.partsupp - | join ON s_suppkey = ps_suppkey myglue_test.tpch_csv.supplier - | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation - | join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region + source = dev.default.partsupp + | join ON s_suppkey = ps_suppkey dev.default.supplier + | join ON s_nationkey = n_nationkey dev.default.nation + | join ON n_regionkey = r_regionkey dev.default.region | where r_name = 'EUROPE' | stats MIN(ps_supplycost) ] | sort - s_acctbal, n_name, s_name, p_partkey | head 100",SUCCESS -"source = myglue_test.tpch_csv.customer -| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders -| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem +"source = dev.default.customer +| join ON c_custkey = o_custkey dev.default.orders +| join ON l_orderkey = o_orderkey dev.default.lineitem | where c_mktsegment = 'BUILDING' AND o_orderdate < date('1995-03-15') AND l_shipdate > date('1995-03-15') | stats sum(l_extendedprice * (1 - l_discount)) as revenue by l_orderkey, o_orderdate, o_shippriority | sort - revenue, o_orderdate | head 10",SUCCESS -"source = myglue_test.tpch_csv.orders +"source = dev.default.orders | where o_orderdate >= date('1993-07-01') and o_orderdate < date_add(date('1993-07-01'), interval 3 month) and exists [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | where l_orderkey = o_orderkey and l_commitdate < l_receiptdate ] | stats count() as order_count by o_orderpriority | sort o_orderpriority",SUCCESS -"source = myglue_test.tpch_csv.customer -| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders -| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem -| join ON l_suppkey = s_suppkey AND c_nationkey = s_nationkey myglue_test.tpch_csv.supplier -| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation -| join ON n_regionkey = r_regionkey myglue_test.tpch_csv.region +"source = dev.default.customer +| join ON c_custkey = o_custkey dev.default.orders +| join ON l_orderkey = o_orderkey dev.default.lineitem +| join ON l_suppkey = s_suppkey AND c_nationkey = s_nationkey dev.default.supplier +| join ON s_nationkey = n_nationkey dev.default.nation +| join ON n_regionkey = r_regionkey dev.default.region | where r_name = 'ASIA' AND o_orderdate >= date('1994-01-01') AND o_orderdate < date_add(date('1994-01-01'), interval 1 year) | stats sum(l_extendedprice * (1 - l_discount)) as revenue by n_name | sort - revenue",SUCCESS -"source = myglue_test.tpch_csv.lineitem +"source = dev.default.lineitem | where l_shipdate >= date('1994-01-01') and l_shipdate < adddate(date('1994-01-01'), 365) and l_discount between .06 - 0.01 and .06 + 0.01 and l_quantity < 24 | stats sum(l_extendedprice * l_discount) as revenue",SUCCESS "source = [ - source = myglue_test.tpch_csv.supplier - | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.lineitem - | join ON o_orderkey = l_orderkey myglue_test.tpch_csv.orders - | join ON c_custkey = o_custkey myglue_test.tpch_csv.customer - | join ON s_nationkey = n1.n_nationkey myglue_test.tpch_csv.nation as n1 - | join ON c_nationkey = n2.n_nationkey myglue_test.tpch_csv.nation as n2 + source = dev.default.supplier + | join ON s_suppkey = l_suppkey dev.default.lineitem + | join ON o_orderkey = l_orderkey dev.default.orders + | join ON c_custkey = o_custkey dev.default.customer + | join ON s_nationkey = n1.n_nationkey dev.default.nation as n1 + | join ON c_nationkey = n2.n_nationkey dev.default.nation as n2 | where l_shipdate between date('1995-01-01') and date('1996-12-31') and n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY' or n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE' | eval supp_nation = n1.n_name, cust_nation = n2.n_name, l_year = year(l_shipdate), volume = l_extendedprice * (1 - l_discount) @@ -338,14 +338,14 @@ by span(age, 10) | sort id | head 10",SUCCESS | stats sum(volume) as revenue by supp_nation, cust_nation, l_year | sort supp_nation, cust_nation, l_year",SUCCESS "source = [ - source = myglue_test.tpch_csv.part - | join ON p_partkey = l_partkey myglue_test.tpch_csv.lineitem - | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.supplier - | join ON l_orderkey = o_orderkey myglue_test.tpch_csv.orders - | join ON o_custkey = c_custkey myglue_test.tpch_csv.customer - | join ON c_nationkey = n1.n_nationkey myglue_test.tpch_csv.nation as n1 - | join ON s_nationkey = n2.n_nationkey myglue_test.tpch_csv.nation as n2 - | join ON n1.n_regionkey = r_regionkey myglue_test.tpch_csv.region + source = dev.default.part + | join ON p_partkey = l_partkey dev.default.lineitem + | join ON s_suppkey = l_suppkey dev.default.supplier + | join ON l_orderkey = o_orderkey dev.default.orders + | join ON o_custkey = c_custkey dev.default.customer + | join ON c_nationkey = n1.n_nationkey dev.default.nation as n1 + | join ON s_nationkey = n2.n_nationkey dev.default.nation as n2 + | join ON n1.n_regionkey = r_regionkey dev.default.region | where r_name = 'AMERICA' AND p_type = 'ECONOMY ANODIZED STEEL' and o_orderdate between date('1995-01-01') and date('1996-12-31') | eval o_year = year(o_orderdate) @@ -358,12 +358,12 @@ by span(age, 10) | sort id | head 10",SUCCESS | fields mkt_share, o_year | sort o_year",SUCCESS "source = [ - source = myglue_test.tpch_csv.part - | join ON p_partkey = l_partkey myglue_test.tpch_csv.lineitem - | join ON s_suppkey = l_suppkey myglue_test.tpch_csv.supplier - | join ON ps_partkey = l_partkey and ps_suppkey = l_suppkey myglue_test.tpch_csv.partsupp - | join ON o_orderkey = l_orderkey myglue_test.tpch_csv.orders - | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation + source = dev.default.part + | join ON p_partkey = l_partkey dev.default.lineitem + | join ON s_suppkey = l_suppkey dev.default.supplier + | join ON ps_partkey = l_partkey and ps_suppkey = l_suppkey dev.default.partsupp + | join ON o_orderkey = l_orderkey dev.default.orders + | join ON s_nationkey = n_nationkey dev.default.nation | where like(p_name, '%green%') | eval nation = n_name | eval o_year = year(o_orderdate) @@ -372,33 +372,33 @@ by span(age, 10) | sort id | head 10",SUCCESS ] as profit | stats sum(amount) as sum_profit by nation, o_year | sort nation, - o_year",SUCCESS -"source = myglue_test.tpch_csv.customer -| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders -| join ON l_orderkey = o_orderkey myglue_test.tpch_csv.lineitem -| join ON c_nationkey = n_nationkey myglue_test.tpch_csv.nation +"source = dev.default.customer +| join ON c_custkey = o_custkey dev.default.orders +| join ON l_orderkey = o_orderkey dev.default.lineitem +| join ON c_nationkey = n_nationkey dev.default.nation | where o_orderdate >= date('1993-10-01') AND o_orderdate < date_add(date('1993-10-01'), interval 3 month) AND l_returnflag = 'R' | stats sum(l_extendedprice * (1 - l_discount)) as revenue by c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment | sort - revenue | head 20",SUCCESS -"source = myglue_test.tpch_csv.partsupp -| join ON ps_suppkey = s_suppkey myglue_test.tpch_csv.supplier -| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation +"source = dev.default.partsupp +| join ON ps_suppkey = s_suppkey dev.default.supplier +| join ON s_nationkey = n_nationkey dev.default.nation | where n_name = 'GERMANY' | stats sum(ps_supplycost * ps_availqty) as value by ps_partkey | where value > [ - source = myglue_test.tpch_csv.partsupp - | join ON ps_suppkey = s_suppkey myglue_test.tpch_csv.supplier - | join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation + source = dev.default.partsupp + | join ON ps_suppkey = s_suppkey dev.default.supplier + | join ON s_nationkey = n_nationkey dev.default.nation | where n_name = 'GERMANY' | stats sum(ps_supplycost * ps_availqty) as check | eval threshold = check * 0.0001000000 | fields threshold ] | sort - value",SUCCESS -"source = myglue_test.tpch_csv.orders -| join ON o_orderkey = l_orderkey myglue_test.tpch_csv.lineitem +"source = dev.default.orders +| join ON o_orderkey = l_orderkey dev.default.lineitem | where l_commitdate < l_receiptdate and l_shipdate < l_commitdate and l_shipmode in ('MAIL', 'SHIP') @@ -409,32 +409,32 @@ by span(age, 10) | sort id | head 10",SUCCESS by l_shipmode | sort l_shipmode",SUCCESS "source = [ - source = myglue_test.tpch_csv.customer + source = dev.default.customer | left outer join ON c_custkey = o_custkey AND not like(o_comment, '%special%requests%') - myglue_test.tpch_csv.orders + dev.default.orders | stats count(o_orderkey) as c_count by c_custkey ] as c_orders | stats count() as custdist by c_count | sort - custdist, - c_count",SUCCESS -"source = myglue_test.tpch_csv.lineitem +"source = dev.default.lineitem | join ON l_partkey = p_partkey AND l_shipdate >= date('1995-09-01') AND l_shipdate < date_add(date('1995-09-01'), interval 1 month) - myglue_test.tpch_csv.part + dev.default.part | stats sum(case(like(p_type, 'PROMO%'), l_extendedprice * (1 - l_discount) else 0)) as sum1, sum(l_extendedprice * (1 - l_discount)) as sum2 | eval promo_revenue = 100.00 * sum1 / sum2 // Stats and Eval commands can combine when issues/819 resolved | fields promo_revenue",SUCCESS -"source = myglue_test.tpch_csv.supplier +"source = dev.default.supplier | join right = revenue0 ON s_suppkey = supplier_no [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | where l_shipdate >= date('1996-01-01') AND l_shipdate < date_add(date('1996-01-01'), interval 3 month) | eval supplier_no = l_suppkey | stats sum(l_extendedprice * (1 - l_discount)) as total_revenue by supplier_no ] | where total_revenue = [ source = [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | where l_shipdate >= date('1996-01-01') AND l_shipdate < date_add(date('1996-01-01'), interval 3 month) | eval supplier_no = l_suppkey | stats sum(l_extendedprice * (1 - l_discount)) as total_revenue by supplier_no @@ -443,24 +443,24 @@ by span(age, 10) | sort id | head 10",SUCCESS ] | sort s_suppkey | fields s_suppkey, s_name, s_address, s_phone, total_revenue",SUCCESS -"source = myglue_test.tpch_csv.partsupp -| join ON p_partkey = ps_partkey myglue_test.tpch_csv.part +"source = dev.default.partsupp +| join ON p_partkey = ps_partkey dev.default.part | where p_brand != 'Brand#45' and not like(p_type, 'MEDIUM POLISHED%') and p_size in (49, 14, 23, 45, 19, 3, 36, 9) and ps_suppkey not in [ - source = myglue_test.tpch_csv.supplier + source = dev.default.supplier | where like(s_comment, '%Customer%Complaints%') | fields s_suppkey ] | stats distinct_count(ps_suppkey) as supplier_cnt by p_brand, p_type, p_size | sort - supplier_cnt, p_brand, p_type, p_size",SUCCESS -"source = myglue_test.tpch_csv.lineitem -| join ON p_partkey = l_partkey myglue_test.tpch_csv.part +"source = dev.default.lineitem +| join ON p_partkey = l_partkey dev.default.part | where p_brand = 'Brand#23' and p_container = 'MED BOX' and l_quantity < [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | where l_partkey = p_partkey | stats avg(l_quantity) as avg | eval `0.2 * avg` = 0.2 * avg @@ -469,11 +469,11 @@ by span(age, 10) | sort id | head 10",SUCCESS | stats sum(l_extendedprice) as sum | eval avg_yearly = sum / 7.0 | fields avg_yearly",SUCCESS -"source = myglue_test.tpch_csv.customer -| join ON c_custkey = o_custkey myglue_test.tpch_csv.orders -| join ON o_orderkey = l_orderkey myglue_test.tpch_csv.lineitem +"source = dev.default.customer +| join ON c_custkey = o_custkey dev.default.orders +| join ON o_orderkey = l_orderkey dev.default.lineitem | where o_orderkey in [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | stats sum(l_quantity) as sum by l_orderkey | where sum > 300 | fields l_orderkey @@ -481,7 +481,7 @@ by span(age, 10) | sort id | head 10",SUCCESS | stats sum(l_quantity) by c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice | sort - o_totalprice, o_orderdate | head 100",SUCCESS -"source = myglue_test.tpch_csv.lineitem +"source = dev.default.lineitem | join ON p_partkey = l_partkey and p_brand = 'Brand#12' and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') @@ -503,19 +503,19 @@ by span(age, 10) | sort id | head 10",SUCCESS and p_size between 1 and 15 and l_shipmode in ('AIR', 'AIR REG') and l_shipinstruct = 'DELIVER IN PERSON' - myglue_test.tpch_csv.part",SUCCESS -"source = myglue_test.tpch_csv.supplier -| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation + dev.default.part",SUCCESS +"source = dev.default.supplier +| join ON s_nationkey = n_nationkey dev.default.nation | where n_name = 'CANADA' and s_suppkey in [ - source = myglue_test.tpch_csv.partsupp + source = dev.default.partsupp | where ps_partkey in [ - source = myglue_test.tpch_csv.part + source = dev.default.part | where like(p_name, 'forest%') | fields p_partkey ] and ps_availqty > [ - source = myglue_test.tpch_csv.lineitem + source = dev.default.lineitem | where l_partkey = ps_partkey and l_suppkey = ps_suppkey and l_shipdate >= date('1994-01-01') @@ -526,19 +526,19 @@ by span(age, 10) | sort id | head 10",SUCCESS ] | fields ps_suppkey ]",SUCCESS -"source = myglue_test.tpch_csv.supplier -| join ON s_suppkey = l1.l_suppkey myglue_test.tpch_csv.lineitem as l1 -| join ON o_orderkey = l1.l_orderkey myglue_test.tpch_csv.orders -| join ON s_nationkey = n_nationkey myglue_test.tpch_csv.nation +"source = dev.default.supplier +| join ON s_suppkey = l1.l_suppkey dev.default.lineitem as l1 +| join ON o_orderkey = l1.l_orderkey dev.default.orders +| join ON s_nationkey = n_nationkey dev.default.nation | where o_orderstatus = 'F' and l1.l_receiptdate > l1.l_commitdate and exists [ - source = myglue_test.tpch_csv.lineitem as l2 + source = dev.default.lineitem as l2 | where l2.l_orderkey = l1.l_orderkey and l2.l_suppkey != l1.l_suppkey ] and not exists [ - source = myglue_test.tpch_csv.lineitem as l3 + source = dev.default.lineitem as l3 | where l3.l_orderkey = l1.l_orderkey and l3.l_suppkey != l1.l_suppkey and l3.l_receiptdate > l3.l_commitdate @@ -548,16 +548,16 @@ by span(age, 10) | sort id | head 10",SUCCESS | sort - numwait, s_name | head 100",SUCCESS "source = [ - source = myglue_test.tpch_csv.customer + source = dev.default.customer | where substring(c_phone, 1, 2) in ('13', '31', '23', '29', '30', '18', '17') and c_acctbal > [ - source = myglue_test.tpch_csv.customer + source = dev.default.customer | where c_acctbal > 0.00 and substring(c_phone, 1, 2) in ('13', '31', '23', '29', '30', '18', '17') | stats avg(c_acctbal) ] and not exists [ - source = myglue_test.tpch_csv.orders + source = dev.default.orders | where o_custkey = c_custkey ] | eval cntrycode = substring(c_phone, 1, 2)