From 7efeab0a098d14022beb221abb095a9848ef7236 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 10:10:02 +0000 Subject: [PATCH 01/42] Initial Spark 4 skeleton --- .../azure-cosmos-spark_4-0_2-13/CHANGELOG.md | 14 + .../CONTRIBUTING.md | 84 ++++++ .../azure-cosmos-spark_4-0_2-13/README.md | 71 +++++ .../azure-cosmos-spark_4-0_2-13/pom.xml | 215 ++++++++++++++ .../scalastyle_config.xml | 130 +++++++++ .../spark/ChangeFeedMicroBatchStream.scala | 271 ++++++++++++++++++ .../spark/CosmosBytesWrittenMetric.scala | 11 + .../azure/cosmos/spark/CosmosCatalog.scala | 59 ++++ .../spark/CosmosRecordsWrittenMetric.scala | 11 + .../cosmos/spark/CosmosRowConverter.scala | 134 +++++++++ .../com/azure/cosmos/spark/CosmosWriter.scala | 109 +++++++ .../com/azure/cosmos/spark/ItemsScan.scala | 41 +++ .../azure/cosmos/spark/ItemsScanBuilder.scala | 137 +++++++++ .../cosmos/spark/ItemsWriterBuilder.scala | 185 ++++++++++++ .../cosmos/spark/RowSerializerPool.scala | 29 ++ .../cosmos/spark/SparkInternalsBridge.scala | 107 +++++++ .../spark/TotalRequestChargeMetric.scala | 11 + ...osmos.spark.CosmosClientBuilderInterceptor | 1 + ...azure.cosmos.spark.CosmosClientInterceptor | 1 + ...cosmos.spark.WriteOnRetryCommitInterceptor | 1 + .../cosmos/spark/CosmosCatalogITest.scala | 97 +++++++ .../cosmos/spark/CosmosRowConverterTest.scala | 97 +++++++ .../azure/cosmos/spark/ItemsScanITest.scala | 256 +++++++++++++++++ .../cosmos/spark/RowSerializerPollTest.scala | 27 ++ .../cosmos/spark/SparkE2EQueryITest.scala | 70 +++++ sdk/cosmos/ci.yml | 15 + sdk/cosmos/pom.xml | 1 + sdk/cosmos/spark.databricks.yml | 5 +- sdk/cosmos/spark.yml | 44 +++ 29 files changed, 2233 insertions(+), 1 deletion(-) create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/CHANGELOG.md create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/CONTRIBUTING.md create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/scalastyle_config.xml create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ChangeFeedMicroBatchStream.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosBytesWrittenMetric.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosCatalog.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRecordsWrittenMetric.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosWriter.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScan.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScanBuilder.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsWriterBuilder.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/RowSerializerPool.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/TotalRequestChargeMetric.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientBuilderInterceptor create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientInterceptor create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.WriteOnRetryCommitInterceptor create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosRowConverterTest.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ItemsScanITest.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/RowSerializerPollTest.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/SparkE2EQueryITest.scala diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CHANGELOG.md b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CHANGELOG.md new file mode 100644 index 000000000000..e3cca105c9e2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CHANGELOG.md @@ -0,0 +1,14 @@ +## Release History + +### 4.43.0-beta.1 (Unreleased) + +#### Features Added +* Initial release of Spark 4.0 connector with Scala 2.13 support + +#### Breaking Changes + +#### Bugs Fixed + +#### Other Changes + +### NOTE: See CHANGELOG.md in 3.3, 3.4, and 3.5 projects for changes in prior Spark versions diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CONTRIBUTING.md b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CONTRIBUTING.md new file mode 100644 index 000000000000..2435e3acead4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/CONTRIBUTING.md @@ -0,0 +1,84 @@ +# Contributing +This instruction is guideline for building and code contribution. + +## Prerequisites +- JDK 17 or above (Spark 4.0 requires Java 17+) +- [Maven](https://maven.apache.org/) 3.0 and above + +## Build from source +To build the project, run maven commands. + +```bash +git clone https://github.com/Azure/azure-sdk-for-java.git +cd sdk/cosmos/azure-cosmos-spark_4-0_2-13 +mvn clean install +``` + +## Test +There are integration tests on azure and on emulator to trigger integration test execution +against Azure Cosmos DB and against +[Azure Cosmos DB Emulator](https://docs.microsoft.com/azure/cosmos-db/local-emulator), you need to +follow the link to set up emulator before test execution. + +- Run unit tests +```bash +mvn clean install -Dgpg.skip +``` + +- Run integration tests + - on Azure + > **NOTE** Please note that integration test against Azure requires Azure Cosmos DB Document + API and will automatically create a Cosmos database in your Azure subscription, then there + will be **Azure usage fee.** + + Integration tests will require a Azure Subscription. If you don't already have an Azure + subscription, you can activate your + [MSDN subscriber benefits](https://azure.microsoft.com/pricing/member-offers/msdn-benefits-details/) + or sign up for a [free Azure account](https://azure.microsoft.com/free/). + + 1. Create an Azure Cosmos DB on Azure. + - Go to [Azure portal](https://portal.azure.com/) and click +New. + - Click Databases, and then click Azure Cosmos DB to create your database. + - Navigate to the database you have created, and click Access keys and copy your + URI and access keys for your database. + + 2. Set environment variables ACCOUNT_HOST, ACCOUNT_KEY and SECONDARY_ACCOUNT_KEY, where value + of them are Cosmos account URI, primary key and secondary key. + + So set the + second group environment variables NEW_ACCOUNT_HOST, NEW_ACCOUNT_KEY and + NEW_SECONDARY_ACCOUNT_KEY, the two group environment variables can be same. + 3. Run maven command with `integration-test-azure` profile. + + ```bash + set ACCOUNT_HOST=your-cosmos-account-uri + set ACCOUNT_KEY=your-cosmos-account-primary-key + set SECONDARY_ACCOUNT_KEY=your-cosmos-account-secondary-key + + set NEW_ACCOUNT_HOST=your-cosmos-account-uri + set NEW_ACCOUNT_KEY=your-cosmos-account-primary-key + set NEW_SECONDARY_ACCOUNT_KEY=your-cosmos-account-secondary-key + mvnw -P integration-test-azure clean install + ``` + + - on Emulator + + Setup Azure Cosmos DB Emulator by following + [this instruction](https://docs.microsoft.com/azure/cosmos-db/local-emulator), and set + associated environment variables. Then run test with: + ```bash + mvnw -P integration-test-emulator install + ``` + + +- Skip tests execution +```bash +mvn clean install -Dgpg.skip -DskipTests +``` + +## Version management +Developing version naming convention is like `0.1.2-beta.1`. Release version naming convention is like `0.1.2`. + +## Contribute to code +Contribution is welcome. Please follow +[this instruction](https://github.com/Azure/azure-sdk-for-java/blob/main/CONTRIBUTING.md) to contribute code. diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md new file mode 100644 index 000000000000..a53b558d4d06 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md @@ -0,0 +1,71 @@ +# Azure Cosmos DB OLTP Spark 4 connector + +## Azure Cosmos DB OLTP Spark 4 connector for Spark 4.0 +**Azure Cosmos DB OLTP Spark connector** provides Apache Spark support for Azure Cosmos DB using +the [SQL API][sql_api_query]. +[Azure Cosmos DB][cosmos_introduction] is a globally-distributed database service which allows +developers to work with data using a variety of standard APIs, such as SQL, MongoDB, Cassandra, Graph, and Table. + +If you have any feedback or ideas on how to improve your experience please let us know here: +https://github.com/Azure/azure-sdk-for-java/issues/new + +### Documentation + +- [Getting started](https://aka.ms/azure-cosmos-spark-3-quickstart) +- [Catalog API](https://aka.ms/azure-cosmos-spark-3-catalog-api) +- [Configuration Parameter Reference](https://aka.ms/azure-cosmos-spark-3-config) + +[//]: # (//TODO: add more sections) +[//]: # (//TODO: Enable Client Logging) +[//]: # (//TODO: Examples) +[//]: # (//TODO: Next steps) +[//]: # (//TODO: Key concepts) +[//]: # (//TODO: Azure Cosmos DB Partition) +[//]: # (//TODO: Troubleshooting) + +### Version Compatibility + +#### azure-cosmos-spark_4-0_2-13 +| Connector | Supported Spark Versions | Minimum Java Version | Supported Scala Versions | Supported Databricks Runtimes | Supported Fabric Runtimes | +|-----------|--------------------------|-----------------------|---------------------------|-------------------------------|---------------------------| +| 4.43.0 | 4.0.0 | [11, 17] | 2.13 | TBD | TBD | + +Note: Spark 4.0 requires Scala 2.13 and Java 11 or higher. When using the Scala API, it is necessary for applications +to use Scala 2.13 that Spark 4.0 was compiled for. + +### Examples +For examples please review our samples repository on [GitHub](https://github.com/Azure-Samples/azure-cosmos-spark-samples). + +### Getting Help +Please create [GitHub issues](https://github.com/Azure/azure-sdk-for-java/issues/new) for any questions and issues. + +### Troubleshooting + +See the [Troubleshooting Guide](docs/troubleshooting.md) for detailed troubleshooting tips. + +### Release Notes +See [CHANGELOG.md](CHANGELOG.md) for detailed release information. + +### Contributing +This project welcomes contributions and suggestions. Most contributions require you to agree to a +Contributor License Agreement (CLA) declaring that you have the right to, and actually do, grant us +the rights to use your contribution. For details, visit https://cla.microsoft.com. + +When you submit a pull request, a CLA-bot will automatically determine whether you need to provide +a CLA and decorate the PR appropriately (e.g., label, comment). Simply follow the instructions +provided by the bot. You will only need to do this once across all repos using our CLA. + +This project has adopted the [Microsoft Open Source Code of Conduct](https://opensource.microsoft.com/codeofconduct/). +For more information see the [Code of Conduct FAQ](https://opensource.microsoft.com/codeofconduct/faq/) or +contact [opencode@microsoft.com](mailto:opencode@microsoft.com) with any additional questions or comments. + +Please refer to our [Contribution guidelines](./CONTRIBUTING.md). + + +[sql_api_query]: https://docs.microsoft.com/azure/cosmos-db/sql-api-sql-query +[cosmos_introduction]: https://docs.microsoft.com/azure/cosmos-db/ +[cosmos_docs]: https://docs.microsoft.com/azure/cosmos-db/introduction +[jdk]: https://docs.microsoft.com/java/azure/jdk/?view=azure-java-stable +[maven]: https://maven.apache.org/ + +![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fcosmos%2Fazure-cosmos-spark_4-0_2-13%2FREADME.png) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml new file mode 100644 index 000000000000..e2b97d65bb1c --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -0,0 +1,215 @@ + + + 4.0.0 + + com.azure.cosmos.spark + azure-cosmos-spark_3 + 0.0.1-beta.1 + ../azure-cosmos-spark_3 + + com.azure.cosmos.spark + azure-cosmos-spark_4-0_2-13 + 4.43.0-beta.1 + jar + https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/cosmos/azure-cosmos-spark_4-0_2-13 + OLTP Spark 4.0 Connector for Azure Cosmos DB SQL API + OLTP Spark 4.0 Connector for Azure Cosmos DB SQL API + + scm:git:https://github.com/Azure/azure-sdk-for-java.git/sdk/cosmos/azure-cosmos-spark_4-0_2-13 + + https://github.com/Azure/azure-sdk-for-java/sdk/cosmos/azure-cosmos-spark_4-0_2-13 + + + Microsoft Corporation + http://microsoft.com + + + + The MIT License (MIT) + http://opensource.org/licenses/MIT + repo + + + + + microsoft + Microsoft Corporation + + + + false + 4.0 + 2.13 + 2.13.17 + 0.9.1 + 0.8.0 + 3.2.2 + 3.2.3 + 3.2.3 + 5.0.0 + true + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 3.6.1 + + + add-sources + generate-sources + + add-source + + + + ${basedir}/../azure-cosmos-spark_3/src/main/scala + ${basedir}/src/main/scala + + + + + add-test-sources + generate-test-sources + + add-test-source + + + + ${basedir}/../azure-cosmos-spark_3/src/test/scala + ${basedir}/src/test/scala + + + + + add-resources + generate-resources + + add-resource + + + + ${basedir}/../azure-cosmos-spark_3/src/main/resources + ${basedir}/src/main/resources + + + + + + + org.apache.maven.plugins + maven-enforcer-plugin + 3.6.1 + + + + + + + spark-e2e_4-0_2-13 + + + ${basedir}/scalastyle_config.xml + + + spark-e2e_4-0_2-13 + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 3.5.3 + + + **/*.* + **/*Test.* + **/*Suite.* + **/*Spec.* + + true + + + + org.scalatest + scalatest-maven-plugin + 2.1.0 + + ${scalatest.argLine} + ${project.build.directory}/surefire-reports + . + SparkTestSuite.txt + (ITest|Test|Spec|Suite) + + + + test + + test + + + + + + + + + java9-plus + + [9,) + + + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + + + + + + org.apache.spark + spark-sql_2.13 + 4.0.0 + + + io.netty + netty-all + + + org.slf4j + * + + + provided + + + org.apache.spark + spark-hive_2.13 + 4.0.0 + + + io.netty + netty-all + + + org.slf4j + * + + + test + + + com.fasterxml.jackson.core + jackson-databind + 2.18.4 + + + com.fasterxml.jackson.module + jackson-module-scala_2.13 + 2.18.4 + + + diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/scalastyle_config.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/scalastyle_config.xml new file mode 100644 index 000000000000..7a8ad2823fb8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/scalastyle_config.xml @@ -0,0 +1,130 @@ + + Scalastyle standard configuration + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ChangeFeedMicroBatchStream.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ChangeFeedMicroBatchStream.scala new file mode 100644 index 000000000000..bf4632cf609a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ChangeFeedMicroBatchStream.scala @@ -0,0 +1,271 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import com.azure.cosmos.changeFeedMetrics.{ChangeFeedMetricsListener, ChangeFeedMetricsTracker} +import com.azure.cosmos.implementation.SparkBridgeImplementationInternal +import com.azure.cosmos.implementation.guava25.collect.{HashBiMap, Maps} +import com.azure.cosmos.spark.CosmosPredicates.{assertNotNull, assertNotNullOrEmpty, assertOnSparkDriver} +import com.azure.cosmos.spark.diagnostics.{DiagnosticsContext, LoggerHelper} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, Offset, ReadLimit, SupportsAdmissionControl} +import org.apache.spark.sql.connector.read.{InputPartition, PartitionReaderFactory} +import org.apache.spark.sql.types.StructType + +import java.time.Duration +import java.util.UUID +import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.atomic.AtomicLong + +// scalastyle:off underscore.import +import scala.collection.JavaConverters._ +// scalastyle:on underscore.import + +// scala style rule flaky - even complaining on partial log messages +// scalastyle:off multiple.string.literals +private class ChangeFeedMicroBatchStream +( + val session: SparkSession, + val schema: StructType, + val config: Map[String, String], + val cosmosClientStateHandles: Broadcast[CosmosClientMetadataCachesSnapshots], + val checkpointLocation: String, + diagnosticsConfig: DiagnosticsConfig +) extends MicroBatchStream + with SupportsAdmissionControl { + + @transient private lazy val log = LoggerHelper.getLogger(diagnosticsConfig, this.getClass) + + private val correlationActivityId = UUID.randomUUID() + private val streamId = correlationActivityId.toString + log.logTrace(s"Instantiated ${this.getClass.getSimpleName}.$streamId") + + private val defaultParallelism = session.sparkContext.defaultParallelism + private val readConfig = CosmosReadConfig.parseCosmosReadConfig(config) + private val sparkEnvironmentInfo = CosmosClientConfiguration.getSparkEnvironmentInfo(Some(session)) + private val clientConfiguration = CosmosClientConfiguration.apply( + config, + readConfig.readConsistencyStrategy, + sparkEnvironmentInfo) + private val containerConfig = CosmosContainerConfig.parseCosmosContainerConfig(config) + private val partitioningConfig = CosmosPartitioningConfig.parseCosmosPartitioningConfig(config) + private val changeFeedConfig = CosmosChangeFeedConfig.parseCosmosChangeFeedConfig(config) + private val clientCacheItem = CosmosClientCache( + clientConfiguration, + Some(cosmosClientStateHandles.value.cosmosClientMetadataCaches), + s"ChangeFeedMicroBatchStream(streamId $streamId)") + private val throughputControlClientCacheItemOpt = + ThroughputControlHelper.getThroughputControlClientCacheItem( + config, clientCacheItem.context, Some(cosmosClientStateHandles), sparkEnvironmentInfo) + private val container = + ThroughputControlHelper.getContainer( + config, + containerConfig, + clientCacheItem, + throughputControlClientCacheItemOpt) + + private var latestOffsetSnapshot: Option[ChangeFeedOffset] = None + + private val partitionIndex = new AtomicLong(0) + private val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + private val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + + if (changeFeedConfig.performanceMonitoringEnabled) { + log.logInfo("ChangeFeed performance monitoring is enabled, registering ChangeFeedMetricsListener") + session.sparkContext.addSparkListener(new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap)) + } else { + log.logInfo("ChangeFeed performance monitoring is disabled") + } + + override def latestOffset(): Offset = { + // For Spark data streams implementing SupportsAdmissionControl trait + // latestOffset(Offset, ReadLimit) is called instead + throw new UnsupportedOperationException( + "latestOffset(Offset, ReadLimit) should be called instead of this method") + } + + /** + * Returns a list of `InputPartition` given the start and end offsets. Each + * `InputPartition` represents a data split that can be processed by one Spark task. The + * number of input partitions returned here is the same as the number of RDD partitions this scan + * outputs. + *

+ * If the `Scan` supports filter push down, this stream is likely configured with a filter + * and is responsible for creating splits for that filter, which is not a full scan. + *

+ *

+ * This method will be called multiple times, to launch one Spark job for each micro-batch in this + * data stream. + *

+ */ + override def planInputPartitions(startOffset: Offset, endOffset: Offset): Array[InputPartition] = { + assertNotNull(startOffset, "startOffset") + assertNotNull(endOffset, "endOffset") + assert(startOffset.isInstanceOf[ChangeFeedOffset], "Argument 'startOffset' is not a change feed offset.") + assert(endOffset.isInstanceOf[ChangeFeedOffset], "Argument 'endOffset' is not a change feed offset.") + + log.logDebug(s"--> planInputPartitions.$streamId, startOffset: ${startOffset.json()} - endOffset: ${endOffset.json()}") + val start = startOffset.asInstanceOf[ChangeFeedOffset] + val end = endOffset.asInstanceOf[ChangeFeedOffset] + + val startChangeFeedState = new String(java.util.Base64.getUrlDecoder.decode(start.changeFeedState)) + log.logDebug(s"Start-ChangeFeedState.$streamId: $startChangeFeedState") + + val endChangeFeedState = new String(java.util.Base64.getUrlDecoder.decode(end.changeFeedState)) + log.logDebug(s"End-ChangeFeedState.$streamId: $endChangeFeedState") + + assert(end.inputPartitions.isDefined, "Argument 'endOffset.inputPartitions' must not be null or empty.") + + val parsedStartChangeFeedState = SparkBridgeImplementationInternal.parseChangeFeedState(start.changeFeedState) + end + .inputPartitions + .get + .map(partition => { + val index = partitionIndexMap.asScala.getOrElseUpdate(partition.feedRange, partitionIndex.incrementAndGet()) + partition + .withContinuationState( + SparkBridgeImplementationInternal + .extractChangeFeedStateForRange(parsedStartChangeFeedState, partition.feedRange), + clearEndLsn = false) + .withIndex(index) + }) + } + + /** + * Returns a factory to create a `PartitionReader` for each `InputPartition`. + */ + override def createReaderFactory(): PartitionReaderFactory = { + log.logDebug(s"--> createReaderFactory.$streamId") + ChangeFeedScanPartitionReaderFactory( + config, + schema, + DiagnosticsContext(correlationActivityId, checkpointLocation), + cosmosClientStateHandles, + diagnosticsConfig, + CosmosClientConfiguration.getSparkEnvironmentInfo(Some(session))) + } + + /** + * Returns the most recent offset available given a read limit. The start offset can be used + * to figure out how much new data should be read given the limit. Users should implement this + * method instead of latestOffset for a MicroBatchStream or getOffset for Source. + * + * When this method is called on a `Source`, the source can return `null` if there is no + * data to process. In addition, for the very first micro-batch, the `startOffset` will be + * null as well. + * + * When this method is called on a MicroBatchStream, the `startOffset` will be `initialOffset` + * for the very first micro-batch. The source can return `null` if there is no data to process. + */ + // This method is doing all the heavy lifting - after calculating the latest offset + // all information necessary to plan partitions is available - so we plan partitions here and + // serialize them in the end offset returned to avoid any IO calls for the actual partitioning + override def latestOffset(startOffset: Offset, readLimit: ReadLimit): Offset = { + + log.logDebug(s"--> latestOffset.$streamId") + + val startChangeFeedOffset = startOffset.asInstanceOf[ChangeFeedOffset] + val offset = CosmosPartitionPlanner.getLatestOffset( + config, + startChangeFeedOffset, + readLimit, + Duration.ZERO, + this.clientConfiguration, + this.cosmosClientStateHandles, + this.containerConfig, + this.partitioningConfig, + this.defaultParallelism, + this.container, + Some(this.partitionMetricsMap) + ) + + if (offset.changeFeedState != startChangeFeedOffset.changeFeedState) { + log.logDebug(s"<-- latestOffset.$streamId - new offset ${offset.json()}") + this.latestOffsetSnapshot = Some(offset) + offset + } else { + log.logDebug(s"<-- latestOffset.$streamId - Finished returning null") + + this.latestOffsetSnapshot = None + + // scalastyle:off null + // null means no more data to process + // null is used here because the DataSource V2 API is defined in Java + null + // scalastyle:on null + } + } + + /** + * Returns the initial offset for a streaming query to start reading from. Note that the + * streaming data source should not assume that it will start reading from its initial offset: + * if Spark is restarting an existing query, it will restart from the check-pointed offset rather + * than the initial one. + */ + // Mapping start form settings to the initial offset/LSNs + override def initialOffset(): Offset = { + assertOnSparkDriver() + + val metadataLog = new ChangeFeedInitialOffsetWriter( + assertNotNull(session, "session"), + assertNotNullOrEmpty(checkpointLocation, "checkpointLocation")) + val offsetJson = metadataLog.get(0).getOrElse { + val newOffsetJson = CosmosPartitionPlanner.createInitialOffset( + container, containerConfig, changeFeedConfig, partitioningConfig, Some(streamId)) + metadataLog.add(0, newOffsetJson) + newOffsetJson + } + + log.logDebug(s"MicroBatch stream $streamId: Initial offset '$offsetJson'.") + ChangeFeedOffset(offsetJson, None) + } + + /** + * Returns the read limits potentially passed to the data source through options when creating + * the data source. + */ + override def getDefaultReadLimit: ReadLimit = { + this.changeFeedConfig.toReadLimit + } + + /** + * Returns the most recent offset available. + * + * The source can return `null`, if there is no data to process or the source does not support + * to this method. + */ + override def reportLatestOffset(): Offset = { + this.latestOffsetSnapshot.orNull + } + + /** + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + */ + override def deserializeOffset(s: String): Offset = { + log.logDebug(s"MicroBatch stream $streamId: Deserialized offset '$s'.") + ChangeFeedOffset.fromJson(s) + } + + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + override def commit(offset: Offset): Unit = { + log.logDebug(s"MicroBatch stream $streamId: Committed offset '${offset.json()}'.") + } + + /** + * Stop this source and free any resources it has allocated. + */ + override def stop(): Unit = { + clientCacheItem.close() + if (throughputControlClientCacheItemOpt.isDefined) { + throughputControlClientCacheItemOpt.get.close() + } + log.logDebug(s"MicroBatch stream $streamId: stopped.") + } +} +// scalastyle:on multiple.string.literals diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosBytesWrittenMetric.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosBytesWrittenMetric.scala new file mode 100644 index 000000000000..9d7f645227bf --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosBytesWrittenMetric.scala @@ -0,0 +1,11 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import org.apache.spark.sql.connector.metric.CustomSumMetric + +private[cosmos] class CosmosBytesWrittenMetric extends CustomSumMetric { + override def name(): String = CosmosConstants.MetricNames.BytesWritten + + override def description(): String = CosmosConstants.MetricNames.BytesWritten +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosCatalog.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosCatalog.scala new file mode 100644 index 000000000000..778c2311e2e0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosCatalog.scala @@ -0,0 +1,59 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException + +import java.util +// scalastyle:off underscore.import +// scalastyle:on underscore.import +import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException} +import org.apache.spark.sql.connector.catalog.{NamespaceChange, SupportsNamespaces} + +// scalastyle:off underscore.import + +class CosmosCatalog + extends CosmosCatalogBase + with SupportsNamespaces { + + override def listNamespaces(): Array[Array[String]] = { + super.listNamespacesBase() + } + + @throws(classOf[NoSuchNamespaceException]) + override def listNamespaces(namespace: Array[String]): Array[Array[String]] = { + super.listNamespacesBase(namespace) + } + + @throws(classOf[NoSuchNamespaceException]) + override def loadNamespaceMetadata(namespace: Array[String]): util.Map[String, String] = { + super.loadNamespaceMetadataBase(namespace) + } + + @throws(classOf[NamespaceAlreadyExistsException]) + override def createNamespace(namespace: Array[String], + metadata: util.Map[String, String]): Unit = { + super.createNamespaceBase(namespace, metadata) + } + + @throws(classOf[UnsupportedOperationException]) + override def alterNamespace(namespace: Array[String], + changes: NamespaceChange*): Unit = { + super.alterNamespaceBase(namespace, changes) + } + + @throws(classOf[NoSuchNamespaceException]) + @throws(classOf[NonEmptyNamespaceException]) + override def dropNamespace(namespace: Array[String], cascade: Boolean): Boolean = { + if (!cascade) { + if (this.listTables(namespace).length > 0) { + throw new NonEmptyNamespaceException(namespace) + } + } + super.dropNamespaceBase(namespace) + } +} +// scalastyle:on multiple.string.literals +// scalastyle:on number.of.methods +// scalastyle:on file.size.limit diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRecordsWrittenMetric.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRecordsWrittenMetric.scala new file mode 100644 index 000000000000..8814c59d0c7d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRecordsWrittenMetric.scala @@ -0,0 +1,11 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import org.apache.spark.sql.connector.metric.CustomSumMetric + +private[cosmos] class CosmosRecordsWrittenMetric extends CustomSumMetric { + override def name(): String = CosmosConstants.MetricNames.RecordsWritten + + override def description(): String = CosmosConstants.MetricNames.RecordsWritten +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala new file mode 100644 index 000000000000..768863a68c88 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala @@ -0,0 +1,134 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import com.azure.cosmos.spark.SchemaConversionModes.SchemaConversionMode +import com.fasterxml.jackson.annotation.JsonInclude.Include +// scalastyle:off underscore.import +import com.fasterxml.jackson.databind.node._ +import com.fasterxml.jackson.databind.{JsonNode, ObjectMapper} +import java.time.format.DateTimeFormatter +import java.time.LocalDateTime +import scala.collection.concurrent.TrieMap + +// scalastyle:off underscore.import +import org.apache.spark.sql.types._ +// scalastyle:on underscore.import + +import scala.util.{Try, Success, Failure} + +// scalastyle:off +private[cosmos] object CosmosRowConverter { + + // TODO: Expose configuration to handle duplicate fields + // See: https://github.com/Azure/azure-sdk-for-java/pull/18642#discussion_r558638474 + private val rowConverterMap = new TrieMap[CosmosSerializationConfig, CosmosRowConverter] + + def get(serializationConfig: CosmosSerializationConfig): CosmosRowConverter = { + rowConverterMap.get(serializationConfig) match { + case Some(existingRowConverter) => existingRowConverter + case None => + val newRowConverterCandidate = createRowConverter(serializationConfig) + rowConverterMap.putIfAbsent(serializationConfig, newRowConverterCandidate) match { + case Some(existingConcurrentlyCreatedRowConverter) => existingConcurrentlyCreatedRowConverter + case None => newRowConverterCandidate + } + } + } + + private def createRowConverter(serializationConfig: CosmosSerializationConfig): CosmosRowConverter = { + val objectMapper = new ObjectMapper() + import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule + objectMapper.registerModule(new JavaTimeModule) + serializationConfig.serializationInclusionMode match { + case SerializationInclusionModes.NonNull => objectMapper.setSerializationInclusion(Include.NON_NULL) + case SerializationInclusionModes.NonEmpty => objectMapper.setSerializationInclusion(Include.NON_EMPTY) + case SerializationInclusionModes.NonDefault => objectMapper.setSerializationInclusion(Include.NON_DEFAULT) + case _ => objectMapper.setSerializationInclusion(Include.ALWAYS) + } + + new CosmosRowConverter(objectMapper, serializationConfig) + } +} + +private[cosmos] class CosmosRowConverter(private val objectMapper: ObjectMapper, private val serializationConfig: CosmosSerializationConfig) + extends CosmosRowConverterBase(objectMapper, serializationConfig) { + + override def convertSparkDataTypeToJsonNodeConditionallyForSparkRuntimeSpecificDataType + ( + fieldType: DataType, + rowData: Any + ): Option[JsonNode] = { + fieldType match { + case TimestampNTZType if rowData.isInstanceOf[java.time.LocalDateTime] => convertToJsonNodeConditionally(rowData.asInstanceOf[java.time.LocalDateTime].toString) + case _ => + throw new Exception(s"Cannot cast $rowData into a Json value. $fieldType has no matching Json value.") + } + } + + override def convertSparkDataTypeToJsonNodeNonNullForSparkRuntimeSpecificDataType(fieldType: DataType, rowData: Any): JsonNode = { + fieldType match { + case TimestampNTZType if rowData.isInstanceOf[java.time.LocalDateTime] => objectMapper.convertValue(rowData.asInstanceOf[java.time.LocalDateTime].toString, classOf[JsonNode]) + case _ => + throw new Exception(s"Cannot cast $rowData into a Json value. $fieldType has no matching Json value.") + } + } + + override def convertToSparkDataTypeForSparkRuntimeSpecificDataType + (dataType: DataType, + value: JsonNode, + schemaConversionMode: SchemaConversionMode): Any = + (value, dataType) match { + case (_, _: TimestampNTZType) => handleConversionErrors(() => toTimestampNTZ(value), schemaConversionMode) + case _ => + throw new IllegalArgumentException( + s"Unsupported datatype conversion [Value: $value] of ${value.getClass}] to $dataType]") + } + + + def toTimestampNTZ(value: JsonNode): LocalDateTime = { + value match { + case isJsonNumber() => LocalDateTime.parse(value.asText()) + case textNode: TextNode => + parseDateTimeNTZFromString(textNode.asText()) match { + case Some(odt) => odt + case None => + throw new IllegalArgumentException( + s"Value '${textNode.asText()} cannot be parsed as LocalDateTime (TIMESTAMP_NTZ).") + } + case _ => LocalDateTime.parse(value.asText()) + } + } + + private def handleConversionErrors[A] = (conversion: () => A, + schemaConversionMode: SchemaConversionMode) => { + Try(conversion()) match { + case Success(convertedValue) => convertedValue + case Failure(error) => + if (schemaConversionMode == SchemaConversionModes.Relaxed) { + null + } + else { + throw error + } + } + } + + def parseDateTimeNTZFromString(value: String): Option[LocalDateTime] = { + try { + val odt = LocalDateTime.parse(value, DateTimeFormatter.ISO_DATE_TIME) + Some(odt) + } + catch { + case _: Exception => + try { + val odt = LocalDateTime.parse(value, DateTimeFormatter.ISO_DATE_TIME) + Some(odt) + } + catch { + case _: Exception => None + } + } + } + +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosWriter.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosWriter.scala new file mode 100644 index 000000000000..042c6ca5636e --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosWriter.scala @@ -0,0 +1,109 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.CosmosDiagnosticsContext +import com.azure.cosmos.implementation.ImplementationBridgeHelpers +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.connector.metric.CustomTaskMetric +import org.apache.spark.sql.connector.write.WriterCommitMessage +import org.apache.spark.sql.execution.metric.CustomMetrics +import org.apache.spark.sql.types.StructType + +import java.util.concurrent.atomic.AtomicLong + +private class CosmosWriter( + userConfig: Map[String, String], + cosmosClientStateHandles: Broadcast[CosmosClientMetadataCachesSnapshots], + diagnosticsConfig: DiagnosticsConfig, + inputSchema: StructType, + partitionId: Int, + taskId: Long, + epochId: Option[Long], + sparkEnvironmentInfo: String) + extends CosmosWriterBase( + userConfig, + cosmosClientStateHandles, + diagnosticsConfig, + inputSchema, + partitionId, + taskId, + epochId, + sparkEnvironmentInfo + ) with OutputMetricsPublisherTrait { + + private val recordsWritten = new AtomicLong(0) + private val bytesWritten = new AtomicLong(0) + private val totalRequestCharge = new AtomicLong(0) + + private val recordsWrittenMetric = new CustomTaskMetric { + override def name(): String = CosmosConstants.MetricNames.RecordsWritten + override def value(): Long = recordsWritten.get() + } + + private val bytesWrittenMetric = new CustomTaskMetric { + override def name(): String = CosmosConstants.MetricNames.BytesWritten + + override def value(): Long = bytesWritten.get() + } + + private val totalRequestChargeMetric = new CustomTaskMetric { + override def name(): String = CosmosConstants.MetricNames.TotalRequestCharge + + // Internally we capture RU/s up to 2 fractional digits to have more precise rounding + override def value(): Long = totalRequestCharge.get() / 100L + } + + private val metrics = Array(recordsWrittenMetric, bytesWrittenMetric, totalRequestChargeMetric) + + override def currentMetricsValues(): Array[CustomTaskMetric] = { + metrics + } + + override def getOutputMetricsPublisher(): OutputMetricsPublisherTrait = this + + override def trackWriteOperation(recordCount: Long, diagnostics: Option[CosmosDiagnosticsContext]): Unit = { + if (recordCount > 0) { + recordsWritten.addAndGet(recordCount) + } + + diagnostics match { + case Some(ctx) => + // Capturing RU/s with 2 fractional digits internally + totalRequestCharge.addAndGet((ctx.getTotalRequestCharge * 100L).toLong) + bytesWritten.addAndGet( + if (ImplementationBridgeHelpers + .CosmosDiagnosticsContextHelper + .getCosmosDiagnosticsContextAccessor + .getOperationType(ctx) + .isReadOnlyOperation) { + + ctx.getMaxRequestPayloadSizeInBytes + ctx.getMaxResponsePayloadSizeInBytes + } else { + ctx.getMaxRequestPayloadSizeInBytes + } + ) + case None => + } + } + + override def commit(): WriterCommitMessage = { + val commitMessage = super.commit() + + // TODO @fabianm - this is a workaround - it shouldn't be necessary to do this here + // Unfortunately WriteToDataSourceV2Exec.scala is not updating custom metrics after the + // call to commit - meaning DataSources which asynchronously write data and flush in commit + // won't get accurate metrics because updates between the last call to write and flushing the + // writes are lost. See https://issues.apache.org/jira/browse/SPARK-45759 + // Once above issue is addressed (probably in Spark 3.4.1 or 3.5 - this needs to be changed + // + // NOTE: This also means that the RU/s metrics cannot be updated in commit - so the + // RU/s metric at the end of a task will be slightly outdated/behind + CustomMetrics.updateMetrics( + currentMetricsValues(), + SparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric(CosmosConstants.MetricNames.KnownCustomMetricNames)) + + commitMessage + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScan.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScan.scala new file mode 100644 index 000000000000..1e193b9e6959 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScan.scala @@ -0,0 +1,41 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.models.PartitionKeyDefinition +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.expressions.NamedReference +import org.apache.spark.sql.connector.read.SupportsRuntimeFiltering +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType + +private[spark] class ItemsScan(session: SparkSession, + schema: StructType, + config: Map[String, String], + readConfig: CosmosReadConfig, + analyzedFilters: AnalyzedAggregatedFilters, + cosmosClientStateHandles: Broadcast[CosmosClientMetadataCachesSnapshots], + diagnosticsConfig: DiagnosticsConfig, + sparkEnvironmentInfo: String, + partitionKeyDefinition: PartitionKeyDefinition) + extends ItemsScanBase( + session, + schema, + config, + readConfig, + analyzedFilters, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo, + partitionKeyDefinition) + with SupportsRuntimeFiltering { // SupportsRuntimeFiltering extends scan + override def filterAttributes(): Array[NamedReference] = { + runtimeFilterAttributesCore() + } + + override def filter(filters: Array[Filter]): Unit = { + runtimeFilterCore(filters) + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScanBuilder.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScanBuilder.scala new file mode 100644 index 000000000000..340a40585eb0 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsScanBuilder.scala @@ -0,0 +1,137 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.SparkBridgeInternal +import com.azure.cosmos.models.PartitionKeyDefinition +import com.azure.cosmos.spark.diagnostics.LoggerHelper +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.connector.read.{Scan, ScanBuilder, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// scalastyle:off underscore.import +import scala.collection.JavaConverters._ +// scalastyle:on underscore.import + +private case class ItemsScanBuilder(session: SparkSession, + config: CaseInsensitiveStringMap, + inputSchema: StructType, + cosmosClientStateHandles: Broadcast[CosmosClientMetadataCachesSnapshots], + diagnosticsConfig: DiagnosticsConfig, + sparkEnvironmentInfo: String) + extends ScanBuilder + with SupportsPushDownFilters + with SupportsPushDownRequiredColumns { + + @transient private lazy val log = LoggerHelper.getLogger(diagnosticsConfig, this.getClass) + log.logTrace(s"Instantiated ${this.getClass.getSimpleName}") + + private val configMap = config.asScala.toMap + private val readConfig = CosmosReadConfig.parseCosmosReadConfig(configMap) + private var processedPredicates : Option[AnalyzedAggregatedFilters] = Option.empty + + private val clientConfiguration = CosmosClientConfiguration.apply( + configMap, + readConfig.readConsistencyStrategy, + CosmosClientConfiguration.getSparkEnvironmentInfo(Some(session)) + ) + private val containerConfig = CosmosContainerConfig.parseCosmosContainerConfig(configMap) + private val description = { + s"""Cosmos ItemsScanBuilder: ${containerConfig.database}.${containerConfig.container}""".stripMargin + } + + private val partitionKeyDefinition: PartitionKeyDefinition = { + TransientErrorsRetryPolicy.executeWithRetry(() => { + val calledFrom = s"ItemsScan($description()).getPartitionKeyDefinition" + Loan( + List[Option[CosmosClientCacheItem]]( + Some(CosmosClientCache.apply( + clientConfiguration, + Some(cosmosClientStateHandles.value.cosmosClientMetadataCaches), + calledFrom + )), + ThroughputControlHelper.getThroughputControlClientCacheItem( + configMap, calledFrom, Some(cosmosClientStateHandles), sparkEnvironmentInfo) + )) + .to(clientCacheItems => { + val container = + ThroughputControlHelper.getContainer( + configMap, + containerConfig, + clientCacheItems(0).get, + clientCacheItems(1)) + + SparkBridgeInternal + .getContainerPropertiesFromCollectionCache(container) + .getPartitionKeyDefinition() + }) + }) + } + + private val filterAnalyzer = FilterAnalyzer(readConfig, partitionKeyDefinition) + + /** + * Pushes down filters, and returns filters that need to be evaluated after scanning. + * @param filters pushed down filters. + * @return the filters that spark need to evaluate + */ + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + this.processedPredicates = Option.apply(filterAnalyzer.analyze(filters)) + + // return the filters that spark need to evaluate + this.processedPredicates.get.filtersNotSupportedByCosmos + } + + /** + * Returns the filters that are pushed to Cosmos as query predicates + * @return filters to be pushed to cosmos db. + */ + override def pushedFilters: Array[Filter] = { + if (this.processedPredicates.isDefined) { + this.processedPredicates.get.filtersToBePushedDownToCosmos + } else { + Array[Filter]() + } + } + + override def build(): Scan = { + val effectiveAnalyzedFilters = this.processedPredicates match { + case Some(analyzedFilters) => analyzedFilters + case None => filterAnalyzer.analyze(Array.empty[Filter]) + } + + // TODO when inferring schema we should consolidate the schema from pruneColumns + new ItemsScan( + session, + inputSchema, + this.configMap, + this.readConfig, + effectiveAnalyzedFilters, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo, + partitionKeyDefinition) + } + + /** + * Applies column pruning w.r.t. the given requiredSchema. + * + * Implementation should try its best to prune the unnecessary columns or nested fields, but it's + * also OK to do the pruning partially, e.g., a data source may not be able to prune nested + * fields, and only prune top-level columns. + * + * Note that, `Scan` implementation should take care of the column + * pruning applied here. + */ + override def pruneColumns(requiredSchema: StructType): Unit = { + // TODO: we need to decide whether do a push down or not on the projection + // spark will do column pruning on the returned data. + // pushing down projection to cosmos has tradeoffs: + // - it increases consumed RU in cosmos query engine + // - it decrease the networking layer latency + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsWriterBuilder.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsWriterBuilder.scala new file mode 100644 index 000000000000..ea759335091b --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/ItemsWriterBuilder.scala @@ -0,0 +1,185 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import com.azure.cosmos.{CosmosAsyncClient, ReadConsistencyStrategy, SparkBridgeInternal} +import com.azure.cosmos.spark.diagnostics.LoggerHelper +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.connector.distributions.{Distribution, Distributions} +import org.apache.spark.sql.connector.expressions.{Expression, Expressions, NullOrdering, SortDirection, SortOrder} +import org.apache.spark.sql.connector.metric.CustomMetric +import org.apache.spark.sql.connector.write.streaming.StreamingWrite +import org.apache.spark.sql.connector.write.{BatchWrite, RequiresDistributionAndOrdering, Write, WriteBuilder} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// scalastyle:off underscore.import +import scala.collection.JavaConverters._ +// scalastyle:on underscore.import + +private class ItemsWriterBuilder +( + userConfig: CaseInsensitiveStringMap, + inputSchema: StructType, + cosmosClientStateHandles: Broadcast[CosmosClientMetadataCachesSnapshots], + diagnosticsConfig: DiagnosticsConfig, + sparkEnvironmentInfo: String +) + extends WriteBuilder { + @transient private lazy val log = LoggerHelper.getLogger(diagnosticsConfig, this.getClass) + log.logTrace(s"Instantiated ${this.getClass.getSimpleName}") + + override def build(): Write = { + new CosmosWrite + } + + override def buildForBatch(): BatchWrite = + new ItemsBatchWriter( + userConfig.asCaseSensitiveMap().asScala.toMap, + inputSchema, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo) + + override def buildForStreaming(): StreamingWrite = + new ItemsBatchWriter( + userConfig.asCaseSensitiveMap().asScala.toMap, + inputSchema, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo) + + private class CosmosWrite extends Write with RequiresDistributionAndOrdering { + + private[this] val supportedCosmosMetrics: Array[CustomMetric] = { + Array( + new CosmosBytesWrittenMetric(), + new CosmosRecordsWrittenMetric(), + new TotalRequestChargeMetric() + ) + } + + // Extract userConfig conversion to avoid repeated calls + private[this] val userConfigMap = userConfig.asCaseSensitiveMap().asScala.toMap + + private[this] val writeConfig = CosmosWriteConfig.parseWriteConfig( + userConfigMap, + inputSchema + ) + + private[this] val containerConfig = CosmosContainerConfig.parseCosmosContainerConfig( + userConfigMap + ) + + override def toBatch(): BatchWrite = + new ItemsBatchWriter( + userConfigMap, + inputSchema, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo) + + override def toStreaming: StreamingWrite = + new ItemsBatchWriter( + userConfigMap, + inputSchema, + cosmosClientStateHandles, + diagnosticsConfig, + sparkEnvironmentInfo) + + override def supportedCustomMetrics(): Array[CustomMetric] = supportedCosmosMetrics + + override def requiredDistribution(): Distribution = { + if (writeConfig.bulkEnabled && writeConfig.bulkTransactional) { + log.logInfo("Transactional batch mode enabled - configuring data distribution by partition key columns") + // For transactional writes, partition by all partition key columns + val partitionKeyPaths = getPartitionKeyColumnNames() + if (partitionKeyPaths.nonEmpty) { + // Use public Expressions.column() factory - returns NamedReference + val clustering = partitionKeyPaths.map(path => Expressions.column(path): Expression).toArray + Distributions.clustered(clustering) + } else { + Distributions.unspecified() + } + } else { + Distributions.unspecified() + } + } + + override def requiredOrdering(): Array[SortOrder] = { + if (writeConfig.bulkEnabled && writeConfig.bulkTransactional) { + // For transactional writes, order by all partition key columns (ascending) + val partitionKeyPaths = getPartitionKeyColumnNames() + if (partitionKeyPaths.nonEmpty) { + partitionKeyPaths.map { path => + // Use public Expressions.sort() factory for creating SortOrder + Expressions.sort( + Expressions.column(path), + SortDirection.ASCENDING, + NullOrdering.NULLS_FIRST + ) + }.toArray + } else { + Array.empty[SortOrder] + } + } else { + Array.empty[SortOrder] + } + } + + private def getPartitionKeyColumnNames(): Seq[String] = { + try { + Loan( + List[Option[CosmosClientCacheItem]]( + Some(createClientForPartitionKeyLookup()) + )) + .to(clientCacheItems => { + val container = ThroughputControlHelper.getContainer( + userConfigMap, + containerConfig, + clientCacheItems(0).get, + None + ) + + // Simplified retrieval using SparkBridgeInternal directly + val containerProperties = SparkBridgeInternal.getContainerPropertiesFromCollectionCache(container) + val partitionKeyDefinition = containerProperties.getPartitionKeyDefinition + + extractPartitionKeyPaths(partitionKeyDefinition) + }) + } catch { + case ex: Exception => + log.logWarning(s"Failed to get partition key definition for transactional writes: ${ex.getMessage}") + Seq.empty[String] + } + } + + private def createClientForPartitionKeyLookup(): CosmosClientCacheItem = { + CosmosClientCache( + CosmosClientConfiguration( + userConfigMap, + ReadConsistencyStrategy.EVENTUAL, + sparkEnvironmentInfo + ), + Some(cosmosClientStateHandles.value.cosmosClientMetadataCaches), + "ItemsWriterBuilder-PKLookup" + ) + } + + private def extractPartitionKeyPaths(partitionKeyDefinition: com.azure.cosmos.models.PartitionKeyDefinition): Seq[String] = { + if (partitionKeyDefinition != null && partitionKeyDefinition.getPaths != null) { + val paths = partitionKeyDefinition.getPaths.asScala + if (paths.isEmpty) { + log.logError("Partition key definition has 0 columns - this should not happen for modern containers") + } + paths.map(path => { + // Remove leading '/' from partition key path (e.g., "/pk" -> "pk") + if (path.startsWith("/")) path.substring(1) else path + }).toSeq + } else { + log.logError("Partition key definition is null - this should not happen for modern containers") + Seq.empty[String] + } + } + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/RowSerializerPool.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/RowSerializerPool.scala new file mode 100644 index 000000000000..427b8757e3e5 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/RowSerializerPool.scala @@ -0,0 +1,29 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.types.StructType + +/** + * Spark serializers are not thread-safe - and expensive to create (dynamic code generation) + * So we will use this object pool to allow reusing serializers based on the targeted schema. + * The main purpose for pooling serializers (vs. creating new ones in each PartitionReader) is for Structured + * Streaming scenarios where PartitionReaders for the same schema could be created every couple of 100 + * milliseconds + * A clean-up task is used to purge serializers for schemas which weren't used anymore + * For each schema we have an object pool that will use a soft-limit to limit the memory footprint + */ +private object RowSerializerPool { + private val serializerFactorySingletonInstance = + new RowSerializerPoolInstance((schema: StructType) => ExpressionEncoder.apply(schema).createSerializer()) + + def getOrCreateSerializer(schema: StructType): ExpressionEncoder.Serializer[Row] = { + serializerFactorySingletonInstance.getOrCreateSerializer(schema) + } + + def returnSerializerToPool(schema: StructType, serializer: ExpressionEncoder.Serializer[Row]): Boolean = { + serializerFactorySingletonInstance.returnSerializerToPool(schema, serializer) + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala new file mode 100644 index 000000000000..45d7bacef995 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/SparkInternalsBridge.scala @@ -0,0 +1,107 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull +import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull +import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait +import org.apache.spark.TaskContext +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.util.AccumulatorV2 + +import java.lang.reflect.Method +import java.util.Locale +import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} +class SparkInternalsBridge { + // Only used in ChangeFeedMetricsListener, which is easier for test validation + def getInternalCustomTaskMetricsAsSQLMetric( + knownCosmosMetricNames: Set[String], + taskMetrics: TaskMetrics) : Map[String, SQLMetric] = { + SparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetricInternal(knownCosmosMetricNames, taskMetrics) + } +} + +object SparkInternalsBridge extends BasicLoggingTrait { + private val SPARK_REFLECTION_ACCESS_ALLOWED_PROPERTY = "COSMOS.SPARK_REFLECTION_ACCESS_ALLOWED" + private val SPARK_REFLECTION_ACCESS_ALLOWED_VARIABLE = "COSMOS_SPARK_REFLECTION_ACCESS_ALLOWED" + + private val DEFAULT_SPARK_REFLECTION_ACCESS_ALLOWED = true + private val accumulatorsMethod : AtomicReference[Method] = new AtomicReference[Method]() + + private def getSparkReflectionAccessAllowed: Boolean = { + val allowedText = System.getProperty( + SPARK_REFLECTION_ACCESS_ALLOWED_PROPERTY, + firstNonNull( + emptyToNull(System.getenv.get(SPARK_REFLECTION_ACCESS_ALLOWED_VARIABLE)), + String.valueOf(DEFAULT_SPARK_REFLECTION_ACCESS_ALLOWED))) + + try { + java.lang.Boolean.valueOf(allowedText.toUpperCase(Locale.ROOT)) + } + catch { + case e: Exception => + logError(s"Parsing spark reflection access allowed $allowedText failed. Using the default $DEFAULT_SPARK_REFLECTION_ACCESS_ALLOWED.", e) + DEFAULT_SPARK_REFLECTION_ACCESS_ALLOWED + } + } + + private final lazy val reflectionAccessAllowed = new AtomicBoolean(getSparkReflectionAccessAllowed) + + def getInternalCustomTaskMetricsAsSQLMetric(knownCosmosMetricNames: Set[String]) : Map[String, SQLMetric] = { + Option.apply(TaskContext.get()) match { + case Some(taskCtx) => getInternalCustomTaskMetricsAsSQLMetric(knownCosmosMetricNames, taskCtx.taskMetrics()) + case None => Map.empty[String, SQLMetric] + } + } + + def getInternalCustomTaskMetricsAsSQLMetric(knownCosmosMetricNames: Set[String], taskMetrics: TaskMetrics) : Map[String, SQLMetric] = { + + if (!reflectionAccessAllowed.get) { + Map.empty[String, SQLMetric] + } else { + getInternalCustomTaskMetricsAsSQLMetricInternal(knownCosmosMetricNames, taskMetrics) + } + } + + private def getAccumulators(taskMetrics: TaskMetrics): Option[Seq[AccumulatorV2[_, _]]] = { + try { + val method = Option(accumulatorsMethod.get) match { + case Some(existing) => existing + case None => + val newMethod = taskMetrics.getClass.getMethod("accumulators") + newMethod.setAccessible(true) + accumulatorsMethod.set(newMethod) + newMethod + } + + val accums = method.invoke(taskMetrics).asInstanceOf[Seq[AccumulatorV2[_, _]]] + + Some(accums) + } catch { + case e: Exception => + logInfo(s"Could not invoke getAccumulators via reflection - Error ${e.getMessage}", e) + + // reflection failed - disabling it for the future + reflectionAccessAllowed.set(false) + None + } + } + + private def getInternalCustomTaskMetricsAsSQLMetricInternal( + knownCosmosMetricNames: Set[String], + taskMetrics: TaskMetrics): Map[String, SQLMetric] = { + getAccumulators(taskMetrics) match { + case Some(accumulators) => accumulators + .filter(accumulable => accumulable.isInstanceOf[SQLMetric] + && accumulable.name.isDefined + && knownCosmosMetricNames.contains(accumulable.name.get)) + .map(accumulable => { + val sqlMetric = accumulable.asInstanceOf[SQLMetric] + sqlMetric.name.get -> sqlMetric + }) + .toMap[String, SQLMetric] + case None => Map.empty[String, SQLMetric] + } + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/TotalRequestChargeMetric.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/TotalRequestChargeMetric.scala new file mode 100644 index 000000000000..56d1f0ba2b78 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/TotalRequestChargeMetric.scala @@ -0,0 +1,11 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import org.apache.spark.sql.connector.metric.CustomSumMetric + +private[cosmos] class TotalRequestChargeMetric extends CustomSumMetric { + override def name(): String = CosmosConstants.MetricNames.TotalRequestCharge + + override def description(): String = CosmosConstants.MetricNames.TotalRequestCharge +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientBuilderInterceptor b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientBuilderInterceptor new file mode 100644 index 000000000000..0d43a5bfc657 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientBuilderInterceptor @@ -0,0 +1 @@ +com.azure.cosmos.spark.TestCosmosClientBuilderInterceptor \ No newline at end of file diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientInterceptor b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientInterceptor new file mode 100644 index 000000000000..e2239720776d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.CosmosClientInterceptor @@ -0,0 +1 @@ +com.azure.cosmos.spark.TestFaultInjectionClientInterceptor \ No newline at end of file diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.WriteOnRetryCommitInterceptor b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.WriteOnRetryCommitInterceptor new file mode 100644 index 000000000000..c60cbf2f14e4 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/resources/META-INF/services/com.azure.cosmos.spark.WriteOnRetryCommitInterceptor @@ -0,0 +1 @@ +com.azure.cosmos.spark.TestWriteOnRetryCommitInterceptor \ No newline at end of file diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala new file mode 100644 index 000000000000..c423992f61cb --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala @@ -0,0 +1,97 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import org.apache.commons.lang3.RandomStringUtils +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException + +class CosmosCatalogITest + extends CosmosCatalogITestBase { + + //scalastyle:off magic.number + + // TODO: spark on windows has issue with this test. + // java.lang.RuntimeException: java.io.IOException: (null) entry in command string: null chmod 0733 D:\tmp\hive; + // once we move Linux CI re-enable the test: + it can "drop an empty database" in { + assume(!Platform.isWindows) + + for (cascade <- Array(true, false)) { + val databaseName = getAutoCleanableDatabaseName + spark.catalog.databaseExists(databaseName) shouldEqual false + + createDatabase(spark, databaseName) + databaseExists(databaseName) shouldEqual true + + dropDatabase(spark, databaseName, cascade) + spark.catalog.databaseExists(databaseName) shouldEqual false + } + } + + it can "drop an non-empty database with cascade true" in { + assume(!Platform.isWindows) + + val databaseName = getAutoCleanableDatabaseName + spark.catalog.databaseExists(databaseName) shouldEqual false + + createDatabase(spark, databaseName) + databaseExists(databaseName) shouldEqual true + + val containerName = RandomStringUtils.randomAlphabetic(5) + spark.sql(s"CREATE TABLE testCatalog.$databaseName.$containerName using cosmos.oltp;") + + dropDatabase(spark, databaseName, true) + spark.catalog.databaseExists(databaseName) shouldEqual false + } + + "drop an non-empty database with cascade false" should "throw NonEmptyNamespaceException" in { + assume(!Platform.isWindows) + + try { + val databaseName = getAutoCleanableDatabaseName + spark.catalog.databaseExists(databaseName) shouldEqual false + + createDatabase(spark, databaseName) + databaseExists(databaseName) shouldEqual true + + val containerName = RandomStringUtils.randomAlphabetic(5) + spark.sql(s"CREATE TABLE testCatalog.$databaseName.$containerName using cosmos.oltp;") + + dropDatabase(spark, databaseName, false) + fail("Expected NonEmptyNamespaceException is not thrown") + } + catch { + case expectedError: NonEmptyNamespaceException => { + logInfo(s"Expected NonEmptyNamespaceException: $expectedError") + succeed + } + } + } + + it can "list all databases" in { + val databaseName1 = getAutoCleanableDatabaseName + val databaseName2 = getAutoCleanableDatabaseName + + // creating those databases ahead of time + cosmosClient.createDatabase(databaseName1).block() + cosmosClient.createDatabase(databaseName2).block() + + val databases = spark.sql("SHOW DATABASES IN testCatalog").collect() + databases.size should be >= 2 + //validate databases has the above database name1 + databases + .filter( + row => row.getAs[String]("namespace").equals(databaseName1) + || row.getAs[String]("namespace").equals(databaseName2)) should have size 2 + } + + private def dropDatabase(spark: SparkSession, databaseName: String, cascade: Boolean) = { + if (cascade) { + spark.sql(s"DROP DATABASE testCatalog.$databaseName CASCADE;") + } else { + spark.sql(s"DROP DATABASE testCatalog.$databaseName;") + } + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosRowConverterTest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosRowConverterTest.scala new file mode 100644 index 000000000000..a5bdc9df94c1 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosRowConverterTest.scala @@ -0,0 +1,97 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait +import com.fasterxml.jackson.databind.ObjectMapper +import com.fasterxml.jackson.databind.node.ObjectNode +import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.types.TimestampNTZType + +import java.sql.{Date, Timestamp} +import java.time.format.DateTimeFormatter +import java.time.{LocalDateTime, OffsetDateTime} + +// scalastyle:off underscore.import +import org.apache.spark.sql.types._ +// scalastyle:on underscore.import + +class CosmosRowConverterTest extends UnitSpec with BasicLoggingTrait { + //scalastyle:off null + //scalastyle:off multiple.string.literals + //scalastyle:off file.size.limit + + val objectMapper = new ObjectMapper() + private[this] val defaultRowConverter = + CosmosRowConverter.get( + new CosmosSerializationConfig( + SerializationInclusionModes.Always, + SerializationDateTimeConversionModes.Default + ) + ) + + + "date and time and TimestampNTZType in spark row" should "translate to ObjectNode" in { + val colName1 = "testCol1" + val colName2 = "testCol2" + val colName3 = "testCol3" + val colName4 = "testCol4" + val currentMillis = System.currentTimeMillis() + val colVal1 = new Date(currentMillis) + val timestampNTZType = "2021-07-01T08:43:28.037" + val colVal2 = LocalDateTime.parse(timestampNTZType, DateTimeFormatter.ISO_DATE_TIME) + val colVal3 = currentMillis.toInt + + val row = new GenericRowWithSchema( + Array(colVal1, colVal2, colVal3, colVal3), + StructType(Seq(StructField(colName1, DateType), + StructField(colName2, TimestampNTZType), + StructField(colName3, DateType), + StructField(colName4, TimestampType)))) + + val objectNode = defaultRowConverter.fromRowToObjectNode(row) + objectNode.get(colName1).asLong() shouldEqual currentMillis + objectNode.get(colName2).asText() shouldEqual "2021-07-01T08:43:28.037" + objectNode.get(colName3).asInt() shouldEqual colVal3 + objectNode.get(colName4).asInt() shouldEqual colVal3 + } + + "time and TimestampNTZType in ObjectNode" should "translate to Row" in { + val colName1 = "testCol1" + val colName2 = "testCol2" + val colName3 = "testCol3" + val colName4 = "testCol4" + val colVal1 = System.currentTimeMillis() + val colVal1AsTime = new Timestamp(colVal1) + val colVal2 = System.currentTimeMillis() + val colVal2AsTime = new Timestamp(colVal2) + val colVal3 = "2021-01-20T20:10:15+01:00" + val colVal3AsTime = Timestamp.valueOf(OffsetDateTime.parse(colVal3, DateTimeFormatter.ISO_OFFSET_DATE_TIME).toLocalDateTime) + val colVal4 = "2021-07-01T08:43:28.037" + val colVal4AsTime = LocalDateTime.parse(colVal4, DateTimeFormatter.ISO_DATE_TIME) + + val objectNode: ObjectNode = objectMapper.createObjectNode() + objectNode.put(colName1, colVal1) + objectNode.put(colName2, colVal2) + objectNode.put(colName3, colVal3) + objectNode.put(colName4, colVal4) + val schema = StructType(Seq( + StructField(colName1, TimestampType), + StructField(colName2, TimestampType), + StructField(colName3, TimestampType), + StructField(colName4, TimestampNTZType))) + val row = defaultRowConverter.fromObjectNodeToRow(schema, objectNode, SchemaConversionModes.Relaxed) + val asTime = row.get(0).asInstanceOf[Timestamp] + asTime.compareTo(colVal1AsTime) shouldEqual 0 + val asTime2 = row.get(1).asInstanceOf[Timestamp] + asTime2.compareTo(colVal2AsTime) shouldEqual 0 + val asTime3 = row.get(2).asInstanceOf[Timestamp] + asTime3.compareTo(colVal3AsTime) shouldEqual 0 + val asTime4 = row.get(3).asInstanceOf[LocalDateTime] + asTime4.compareTo(colVal4AsTime) shouldEqual 0 + } + + //scalastyle:on null + //scalastyle:on multiple.string.literals + //scalastyle:on file.size.limit +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ItemsScanITest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ItemsScanITest.scala new file mode 100644 index 000000000000..b6433c6d7b2f --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ItemsScanITest.scala @@ -0,0 +1,256 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.implementation.{CosmosClientMetadataCachesSnapshot, SparkBridgeImplementationInternal, TestConfigurations, Utils} +import com.azure.cosmos.models.PartitionKey +import com.fasterxml.jackson.databind.node.ObjectNode +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.connector.expressions.Expressions +import org.apache.spark.sql.sources.{Filter, In} +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +import java.util.UUID +import scala.collection.mutable.ListBuffer + +class ItemsScanITest + extends IntegrationSpec + with Spark + with AutoCleanableCosmosContainersWithPkAsPartitionKey { + + //scalastyle:off multiple.string.literals + //scalastyle:off magic.number + + private val idProperty = "id" + private val pkProperty = "pk" + private val itemIdentityProperty = "_itemIdentity" + + private val analyzedAggregatedFilters = + AnalyzedAggregatedFilters( + QueryFilterAnalyzer.rootParameterizedQuery, + false, + Array.empty[Filter], + Array.empty[Filter], + Option.empty[List[ReadManyFilter]]) + + it should "only return readMany filtering property when runtTimeFiltering is enabled and readMany filtering is enabled" in { + val clientMetadataCachesSnapshots = getCosmosClientMetadataCachesSnapshots() + + val testCases = Array( + // containerName, partitionKey property, expected readMany filtering property + (cosmosContainer, idProperty, idProperty), + (cosmosContainersWithPkAsPartitionKey, pkProperty, itemIdentityProperty) + ) + + for (testCase <- testCases) { + val partitionKeyDefinition = + cosmosClient + .getDatabase(cosmosDatabase) + .getContainer(testCase._1) + .read() + .block() + .getProperties + .getPartitionKeyDefinition + + for (runTimeFilteringEnabled <- Array(true, false)) { + for (readManyFilteringEnabled <- Array(true, false)) { + logInfo(s"TestCase: containerName ${testCase._1}, partitionKeyProperty ${testCase._2}, " + + s"runtimeFilteringEnabled $runTimeFilteringEnabled, readManyFilteringEnabled $readManyFilteringEnabled") + + val config = Map( + "spark.cosmos.accountEndpoint" -> TestConfigurations.HOST, + "spark.cosmos.accountKey" -> TestConfigurations.MASTER_KEY, + "spark.cosmos.database" -> cosmosDatabase, + "spark.cosmos.container" -> testCase._1, + "spark.cosmos.read.inferSchema.enabled" -> "true", + "spark.cosmos.applicationName" -> "ItemsScan", + "spark.cosmos.read.runtimeFiltering.enabled" -> runTimeFilteringEnabled.toString, + "spark.cosmos.read.readManyFiltering.enabled" -> readManyFilteringEnabled.toString + ) + val readConfig = CosmosReadConfig.parseCosmosReadConfig(config) + val diagnosticsConfig = DiagnosticsConfig.parseDiagnosticsConfig(config) + val schema = getDefaultSchema(testCase._2) + + val itemScan = new ItemsScan( + spark, + schema, + config, + readConfig, + analyzedAggregatedFilters, + clientMetadataCachesSnapshots, + diagnosticsConfig, + "", + partitionKeyDefinition) + val arrayReferences = itemScan.filterAttributes() + + if (runTimeFilteringEnabled && readManyFilteringEnabled) { + arrayReferences.size shouldBe 1 + arrayReferences should contain theSameElementsAs Array(Expressions.column(testCase._3)) + } else { + arrayReferences shouldBe empty + } + } + } + } + } + + it should "only prune partitions when runtTimeFiltering is enabled and readMany filtering is enabled" in { + val clientMetadataCachesSnapshots = getCosmosClientMetadataCachesSnapshots() + + val testCases = Array( + //containerName, partitionKeyProperty, expected readManyFiltering property + (cosmosContainer, idProperty, idProperty), + (cosmosContainersWithPkAsPartitionKey, pkProperty, itemIdentityProperty) + ) + for (testCase <- testCases) { + val container = cosmosClient.getDatabase(cosmosDatabase).getContainer(testCase._1) + val partitionKeyDefinition = container.read().block().getProperties.getPartitionKeyDefinition + + // assert that there is more than one range + val feedRanges = container.getFeedRanges.block() + feedRanges.size() should be > 1 + + // first inject few items + val matchingItemList = ListBuffer[ObjectNode]() + for (_ <- 1 to 20) { + val objectNode = getNewItem(testCase._2) + container.createItem(objectNode).block() + matchingItemList += objectNode + logInfo(s"ID of test doc: ${objectNode.get(idProperty).asText()}") + } + + // choose one of the items created above and filter by it + val runtimeFilters = getReadManyFilters(Array(matchingItemList(0)), testCase._2, testCase._3) + + for (runTimeFilteringEnabled <- Array(true, false)) { + for (readManyFilteringEnabled <- Array(true, false)) { + logInfo(s"TestCase: containerName ${testCase._1}, partitionKeyProperty ${testCase._2}, " + + s"runtimeFilteringEnabled $runTimeFilteringEnabled, readManyFilteringEnabled $readManyFilteringEnabled") + + val config = Map( + "spark.cosmos.accountEndpoint" -> TestConfigurations.HOST, + "spark.cosmos.accountKey" -> TestConfigurations.MASTER_KEY, + "spark.cosmos.database" -> cosmosDatabase, + "spark.cosmos.container" -> testCase._1, + "spark.cosmos.read.inferSchema.enabled" -> "true", + "spark.cosmos.applicationName" -> "ItemsScan", + "spark.cosmos.read.partitioning.strategy" -> "Restrictive", + "spark.cosmos.read.runtimeFiltering.enabled" -> runTimeFilteringEnabled.toString, + "spark.cosmos.read.readManyFiltering.enabled" -> readManyFilteringEnabled.toString + ) + val readConfig = CosmosReadConfig.parseCosmosReadConfig(config) + val diagnosticsConfig = DiagnosticsConfig.parseDiagnosticsConfig(config) + + val schema = getDefaultSchema(testCase._2) + val itemScan = new ItemsScan( + spark, + schema, + config, + readConfig, + analyzedAggregatedFilters, + clientMetadataCachesSnapshots, + diagnosticsConfig, + "", + partitionKeyDefinition) + + val plannedInputPartitions = itemScan.planInputPartitions() + plannedInputPartitions.length shouldBe feedRanges.size() // using restrictive strategy + + itemScan.filter(runtimeFilters) + val plannedInputPartitionAfterFiltering = itemScan.planInputPartitions() + + if (runTimeFilteringEnabled && readManyFilteringEnabled) { + // partition can be pruned + plannedInputPartitionAfterFiltering.length shouldBe 1 + val filterItemFeedRange = + SparkBridgeImplementationInternal.partitionKeyToNormalizedRange( + new PartitionKey(getPartitionKeyValue(matchingItemList(0), s"/${testCase._2}")), + partitionKeyDefinition) + + val rangesOverlap = + SparkBridgeImplementationInternal.doRangesOverlap( + filterItemFeedRange, + plannedInputPartitionAfterFiltering(0).asInstanceOf[CosmosInputPartition].feedRange) + + rangesOverlap shouldBe true + } else { + // no partition will be pruned + plannedInputPartitionAfterFiltering.length shouldBe plannedInputPartitions.length + plannedInputPartitionAfterFiltering should contain theSameElementsAs plannedInputPartitions + } + } + } + } + } + + private def getCosmosClientMetadataCachesSnapshots(): Broadcast[CosmosClientMetadataCachesSnapshots] = { + val cosmosClientMetadataCachesSnapshot = new CosmosClientMetadataCachesSnapshot() + cosmosClientMetadataCachesSnapshot.serialize(cosmosClient) + + spark.sparkContext.broadcast( + CosmosClientMetadataCachesSnapshots( + cosmosClientMetadataCachesSnapshot, + Option.empty[CosmosClientMetadataCachesSnapshot])) + } + + private def getReadManyFilters( + filteringItems: Array[ObjectNode], + partitionKeyProperty: String, + readManyFilteringProperty: String): Array[Filter] = { + val readManyFilterValues = + filteringItems + .map(filteringItem => getReadManyFilteringValue(filteringItem, partitionKeyProperty, readManyFilteringProperty)) + + if (partitionKeyProperty.equalsIgnoreCase(idProperty)) { + Array[Filter](In(idProperty, readManyFilterValues.map(_.asInstanceOf[Any]))) + } else { + Array[Filter](In(readManyFilteringProperty, readManyFilterValues.map(_.asInstanceOf[Any]))) + } + } + + private def getReadManyFilteringValue( + objectNode: ObjectNode, + partitionKeyProperty: String, + readManyFilteringProperty: String): String = { + + if (readManyFilteringProperty.equals(itemIdentityProperty)) { + CosmosItemIdentityHelper + .getCosmosItemIdentityValueString( + objectNode.get(idProperty).asText(), + List(objectNode.get(partitionKeyProperty).asText())) + } else { + objectNode.get(idProperty).asText() + } + } + + private def getNewItem(partitionKeyProperty: String): ObjectNode = { + val objectNode = Utils.getSimpleObjectMapper.createObjectNode() + val id = UUID.randomUUID().toString + objectNode.put(idProperty, id) + + if (!partitionKeyProperty.equalsIgnoreCase(idProperty)) { + val pk = UUID.randomUUID().toString + objectNode.put(partitionKeyProperty, pk) + } + + objectNode + } + + private def getDefaultSchema(partitionKeyProperty: String): StructType = { + if (!partitionKeyProperty.equalsIgnoreCase(idProperty)) { + StructType(Seq( + StructField(idProperty, StringType), + StructField(pkProperty, StringType), + StructField(itemIdentityProperty, StringType) + )) + } else { + StructType(Seq( + StructField(idProperty, StringType) + )) + } + } + + //scalastyle:on multiple.string.literals + //scalastyle:on magic.number +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/RowSerializerPollTest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/RowSerializerPollTest.scala new file mode 100644 index 000000000000..2335bedf917d --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/RowSerializerPollTest.scala @@ -0,0 +1,27 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. +package com.azure.cosmos.spark + +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} + +class RowSerializerPollTest extends RowSerializerPollSpec { + //scalastyle:off multiple.string.literals + + "RowSerializer " should "be returned to the pool only a limited number of times" in { + val canRun = Platform.canRunTestAccessingDirectByteBuffer + assume(canRun._1, canRun._2) + + val schema = StructType(Seq(StructField("column01", IntegerType), StructField("column02", StringType))) + + for (_ <- 1 to 256) { + RowSerializerPool.returnSerializerToPool(schema, ExpressionEncoder.apply(schema).createSerializer()) shouldBe true + } + + logInfo("First 256 attempt to pool succeeded") + + RowSerializerPool.returnSerializerToPool(schema, ExpressionEncoder.apply(schema).createSerializer()) shouldBe false + } + //scalastyle:on multiple.string.literals +} + diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/SparkE2EQueryITest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/SparkE2EQueryITest.scala new file mode 100644 index 000000000000..5f9cb1dbdbc8 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/SparkE2EQueryITest.scala @@ -0,0 +1,70 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.implementation.TestConfigurations +import com.fasterxml.jackson.databind.node.ObjectNode + +import java.util.UUID + +class SparkE2EQueryITest + extends SparkE2EQueryITestBase { + + "spark query" can "return proper Cosmos specific query plan on explain with nullable properties" in { + val cosmosEndpoint = TestConfigurations.HOST + val cosmosMasterKey = TestConfigurations.MASTER_KEY + + val id = UUID.randomUUID().toString + + val rawItem = + s""" + | { + | "id" : "$id", + | "nestedObject" : { + | "prop1" : 5, + | "prop2" : "6" + | } + | } + |""".stripMargin + + val objectNode = objectMapper.readValue(rawItem, classOf[ObjectNode]) + + val container = cosmosClient.getDatabase(cosmosDatabase).getContainer(cosmosContainer) + container.createItem(objectNode).block() + + val cfg = Map("spark.cosmos.accountEndpoint" -> cosmosEndpoint, + "spark.cosmos.accountKey" -> cosmosMasterKey, + "spark.cosmos.database" -> cosmosDatabase, + "spark.cosmos.container" -> cosmosContainer, + "spark.cosmos.read.inferSchema.forceNullableProperties" -> "true", + "spark.cosmos.read.partitioning.strategy" -> "Restrictive" + ) + + val df = spark.read.format("cosmos.oltp").options(cfg).load() + val rowsArray = df.where("nestedObject.prop2 = '6'").collect() + rowsArray should have size 1 + + var output = new java.io.ByteArrayOutputStream() + Console.withOut(output) { + df.explain() + } + var queryPlan = output.toString.replaceAll("#\\d+", "#x") + logInfo(s"Query Plan: $queryPlan") + queryPlan.contains("Cosmos Query: SELECT * FROM r") shouldEqual true + + output = new java.io.ByteArrayOutputStream() + Console.withOut(output) { + df.where("nestedObject.prop2 = '6'").explain() + } + queryPlan = output.toString.replaceAll("#\\d+", "#x") + logInfo(s"Query Plan: $queryPlan") + val expected = s"Cosmos Query: SELECT * FROM r WHERE (NOT(IS_NULL(r['nestedObject']['prop2'])) AND IS_DEFINED(r['nestedObject']['prop2'])) " + + s"AND r['nestedObject']['prop2']=" + + s"@param0${System.getProperty("line.separator")} > param: @param0 = 6" + queryPlan.contains(expected) shouldEqual true + + val item = rowsArray(0) + item.getAs[String]("id") shouldEqual id + } +} diff --git a/sdk/cosmos/ci.yml b/sdk/cosmos/ci.yml index 2e1c845dae37..0433113ce468 100644 --- a/sdk/cosmos/ci.yml +++ b/sdk/cosmos/ci.yml @@ -19,6 +19,7 @@ trigger: - sdk/cosmos/azure-cosmos-spark_3-5/ - sdk/cosmos/azure-cosmos-spark_3-5_2-12/ - sdk/cosmos/azure-cosmos-spark_3-5_2-13/ + - sdk/cosmos/azure-cosmos-spark_4-0_2-13/ - sdk/cosmos/fabric-cosmos-spark-auth_3/ - sdk/cosmos/azure-cosmos-test/ - sdk/cosmos/azure-cosmos-tests/ @@ -36,6 +37,7 @@ trigger: - sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml - sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml - sdk/cosmos/azure-cosmos-spark_3-5/pom.xml + - sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml - sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml - sdk/cosmos/azure-cosmos-kafka-connect/pom.xml @@ -62,6 +64,7 @@ pr: - sdk/cosmos/azure-cosmos-spark_3-5/ - sdk/cosmos/azure-cosmos-spark_3-5_2-12/ - sdk/cosmos/azure-cosmos-spark_3-5_2-13/ + - sdk/cosmos/azure-cosmos-spark_4-0_2-13/ - sdk/cosmos/fabric-cosmos-spark-auth_3/ - sdk/cosmos/faq/ - sdk/cosmos/azure-cosmos-kafka-connect/ @@ -76,6 +79,7 @@ pr: - sdk/cosmos/azure-cosmos-spark_3-5/pom.xml - sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml - sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml + - sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml - sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml - sdk/cosmos/azure-cosmos-test/pom.xml - sdk/cosmos/azure-cosmos-tests/pom.xml @@ -105,6 +109,10 @@ parameters: displayName: 'azure-cosmos-spark_3-5_2-13' type: boolean default: true + - name: release_azurecosmosspark40_scala213 + displayName: 'azure-cosmos-spark_4-0_2-13' + type: boolean + default: true - name: release_fabriccosmossparkauth3 displayName: 'fabric-cosmos-spark-auth_3' type: boolean @@ -160,6 +168,13 @@ extends: skipPublishDocGithubIo: true skipPublishDocMs: true releaseInBatch: ${{ parameters.release_azurecosmosspark35_scala213 }} + - name: azure-cosmos-spark_4-0_2-13 + groupId: com.azure.cosmos.spark + safeName: azurecosmosspark40scala213 + uberJar: true + skipPublishDocGithubIo: true + skipPublishDocMs: true + releaseInBatch: ${{ parameters.release_azurecosmosspark40_scala213 }} - name: fabric-cosmos-spark-auth_3 groupId: com.azure.cosmos.spark safeName: fabriccosmossparkauth3 diff --git a/sdk/cosmos/pom.xml b/sdk/cosmos/pom.xml index 81b2db7ca53e..69f77543edb3 100644 --- a/sdk/cosmos/pom.xml +++ b/sdk/cosmos/pom.xml @@ -19,6 +19,7 @@ azure-cosmos-spark_3-5 azure-cosmos-spark_3-5_2-12 azure-cosmos-spark_3-5_2-13 + azure-cosmos-spark_4-0_2-13 azure-cosmos-test azure-cosmos-tests azure-cosmos-kafka-connect diff --git a/sdk/cosmos/spark.databricks.yml b/sdk/cosmos/spark.databricks.yml index e7044941e9c9..6b62dfbefe06 100644 --- a/sdk/cosmos/spark.databricks.yml +++ b/sdk/cosmos/spark.databricks.yml @@ -38,6 +38,9 @@ parameters: type: string - name: JarName type: string + - name: JavaVersion + type: string + default: '1.11' stages: - stage: displayName: 'Spark Databricks integration ${{ parameters.ClusterName }} - ${{ parameters.SparkVersion }}' @@ -59,7 +62,7 @@ stages: goals: 'package' options: '$(DefaultOptions) -T 1 -Ppackage-assembly -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Drevapi.skip=true -pl com.azure:azure-cosmos,com.azure:azure-cosmos-test,com.azure.cosmos.spark:${{ parameters.SparkVersion }}' javaHomeOption: 'JDKVersion' - jdkVersionOption: '1.11' + jdkVersionOption: '${{ parameters.JavaVersion }}' jdkArchitectureOption: 'x64' publishJUnitResults: false - task: UsePythonVersion@0 diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index 0dcfd657d0ae..e210c46338ac 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -111,3 +111,47 @@ stages: JarReadOnlySasUri: $(spark-databricks-token) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-13-latest-ci-candidate.jar' + - template: /sdk/cosmos/spark.databricks.yml + parameters: + CosmosEndpointMsi: $(spark-databricks-cosmos-endpoint-msi) + CosmosEndpoint: $(spark-databricks-cosmos-endpoint) + CosmosKey: $(spark-databricks-cosmos-key) + DatabricksEndpoint: $(spark-databricks-endpoint-with-msi) + SubscriptionId: '8fba6d4f-7c37-4d13-9063-fd58ad2b86e2' + TenantId: '72f988bf-86f1-41af-91ab-2d7cd011db47' + ResourceGroupName: 'oltp-spark-ci' + ClientId: $(spark-databricks-cosmos-spn-clientId) + ClientSecret: $(spark-databricks-cosmos-spn-clientSecret) + CosmosContainerName: 'sampleContainer8' + CosmosDatabaseName: 'sampleDB8' + DatabricksToken: $(spark-databricks-token-with-msi) + SparkVersion: 'azure-cosmos-spark_4-0_2-13' + ClusterName: 'oltp-ci-spark40-2workers-ds3v2-scala_2.13' + AvoidDBFS: true + JavaVersion: '1.21' + JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) + JarReadOnlySasUri: $(spark-databricks-token) + JarStorageAccountName: 'oltpsparkcijarstore1225' + JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' + - template: /sdk/cosmos/spark.databricks.yml + parameters: + CosmosEndpointMsi: $(spark-databricks-cosmos-endpoint-msi) + CosmosEndpoint: $(spark-databricks-cosmos-endpoint) + CosmosKey: $(spark-databricks-cosmos-key) + DatabricksEndpoint: $(spark-databricks-endpoint-with-msi) + SubscriptionId: '8fba6d4f-7c37-4d13-9063-fd58ad2b86e2' + TenantId: '72f988bf-86f1-41af-91ab-2d7cd011db47' + ResourceGroupName: 'oltp-spark-ci' + ClientId: $(spark-databricks-cosmos-spn-clientId) + ClientSecret: $(spark-databricks-cosmos-spn-clientSecret) + CosmosContainerName: 'sampleContainer9' + CosmosDatabaseName: 'sampleDB9' + DatabricksToken: $(spark-databricks-token-with-msi) + SparkVersion: 'azure-cosmos-spark_4-0_2-13' + ClusterName: 'oltp-ci-spark40-2workers-ds3v2-scala_2.13' + AvoidDBFS: true + JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) + JarReadOnlySasUri: $(spark-databricks-token) + JarStorageAccountName: 'oltpsparkcijarstore1225' + JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' + JavaVersion: '1.21' From 34744fcc26b9fba3095326fbf1dea39f3c6a10e3 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 11:01:35 +0000 Subject: [PATCH 02/42] Small fixes --- eng/versioning/external_dependencies.txt | 2 ++ sdk/cosmos/azure-cosmos-spark_3/dev/README.md | 1 + sdk/cosmos/azure-cosmos-spark_3/pom.xml | 1 + 3 files changed, 4 insertions(+) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 7786b0c40ddb..59d308115060 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -248,9 +248,11 @@ cosmos_com.microsoft.azure.synapse:synapseutils_2.12;1.5.4 cosmos-spark_3-3_org.apache.spark:spark-sql_2.12;3.3.0 cosmos-spark_3-4_org.apache.spark:spark-sql_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-sql_2.12;3.5.0 +cosmos-spark_4-0_org.apache.spark:spark-sql_2.13;4.0.0 cosmos-spark_3-3_org.apache.spark:spark-hive_2.12;3.3.0 cosmos-spark_3-4_org.apache.spark:spark-hive_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-hive_2.12;3.5.0 +cosmos-spark_4-0_org.apache.spark:spark-hive_2.13;4.0.0 cosmos_org.scala-lang:scala-library;2.12.19 cosmos-scala213_org.scala-lang:scala-library;2.13.17 cosmos_org.scala-lang.modules:scala-java8-compat_2.12;0.9.1 diff --git a/sdk/cosmos/azure-cosmos-spark_3/dev/README.md b/sdk/cosmos/azure-cosmos-spark_3/dev/README.md index 30ba9fbfc0db..a7b7c1e0ca9e 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/dev/README.md +++ b/sdk/cosmos/azure-cosmos-spark_3/dev/README.md @@ -47,6 +47,7 @@ mvn -e -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true mvn -e -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -pl ,azure-cosmos-spark_3-4_2-12 clean install mvn -e -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -pl ,azure-cosmos-spark_3-5_2-12 clean install mvn -e -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -pl ,azure-cosmos-spark_3-5_2-13 clean install +mvn -e -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dspotbugs.skip=true -Dcheckstyle.skip=true -Drevapi.skip=true -pl ,azure-cosmos-spark_4-0_2-13 clean install ``` Take these files: diff --git a/sdk/cosmos/azure-cosmos-spark_3/pom.xml b/sdk/cosmos/azure-cosmos-spark_3/pom.xml index 3dc770578e67..0446de1aa31e 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3/pom.xml @@ -296,6 +296,7 @@ org.apache.spark:spark-sql_2.12:[${spark34.version}] org.apache.spark:spark-sql_2.12:[${spark35.version}] org.apache.spark:spark-sql_2.13:[${spark35.version}] + org.apache.spark:spark-sql_2.13:[4.0.0] org.scala-lang:scala-library:[${scala.version}] org.scala-lang.modules:scala-java8-compat_2.12:[${scala-java8-compat.version}] org.scala-lang.modules:scala-java8-compat_2.13:[${scala-java8-compat.version}] From 030f42c1d6b0a612e88f935f3c81481a7b344ce5 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 11:16:44 +0000 Subject: [PATCH 03/42] Update cosmos-emulator-matrix.json --- .../templates/stages/cosmos-emulator-matrix.json | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index d1a2da8a0c64..2823975d5ea8 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -92,6 +92,20 @@ "JavaTestVersion": "1.17", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, + "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.21", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { "ProfileFlag": "-Pkafka-emulator", "PROTOCOLS": "[\"Tcp\"]", From 2e2fc170959a8f8c6baeae76f71999f3834f9339 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 11:23:36 +0000 Subject: [PATCH 04/42] Reacted to code review comments --- sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md | 8 ++++---- .../com/azure/cosmos/spark/CosmosRowConverter.scala | 9 +-------- sdk/cosmos/spark.yml | 2 +- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md index a53b558d4d06..1ed8a8186288 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md @@ -26,11 +26,11 @@ https://github.com/Azure/azure-sdk-for-java/issues/new ### Version Compatibility #### azure-cosmos-spark_4-0_2-13 -| Connector | Supported Spark Versions | Minimum Java Version | Supported Scala Versions | Supported Databricks Runtimes | Supported Fabric Runtimes | -|-----------|--------------------------|-----------------------|---------------------------|-------------------------------|---------------------------| -| 4.43.0 | 4.0.0 | [11, 17] | 2.13 | TBD | TBD | +| Connector | Supported Spark Versions | Minimum Java Version | Supported Scala Versions | Supported Databricks Runtimes | Supported Fabric Runtimes | +|-----------|--------------------------|----------------------|---------------------------|-------------------------------|---------------------------| +| 4.43.0 | 4.0.0 | [17, 21] | 2.13 | TBD | TBD | -Note: Spark 4.0 requires Scala 2.13 and Java 11 or higher. When using the Scala API, it is necessary for applications +Note: Spark 4.0 requires Scala 2.13 and Java 17 or higher. When using the Scala API, it is necessary for applications to use Scala 2.13 that Spark 4.0 was compiled for. ### Examples diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala index 768863a68c88..fb4e9db760a0 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/scala/com/azure/cosmos/spark/CosmosRowConverter.scala @@ -120,14 +120,7 @@ private[cosmos] class CosmosRowConverter(private val objectMapper: ObjectMapper, Some(odt) } catch { - case _: Exception => - try { - val odt = LocalDateTime.parse(value, DateTimeFormatter.ISO_DATE_TIME) - Some(odt) - } - catch { - case _: Exception => None - } + case _: Exception => None } } diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index e210c46338ac..e20eeecc2ea0 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -150,8 +150,8 @@ stages: SparkVersion: 'azure-cosmos-spark_4-0_2-13' ClusterName: 'oltp-ci-spark40-2workers-ds3v2-scala_2.13' AvoidDBFS: true + JavaVersion: '1.21' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-token) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' - JavaVersion: '1.21' From 3dfd45298c190a25b4d50a5ea1b17cf940e5876e Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 11:40:15 +0000 Subject: [PATCH 05/42] Update spark.yml --- sdk/cosmos/spark.yml | 24 +----------------------- 1 file changed, 1 insertion(+), 23 deletions(-) diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index e20eeecc2ea0..e2ed7791ced5 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -126,29 +126,7 @@ stages: CosmosDatabaseName: 'sampleDB8' DatabricksToken: $(spark-databricks-token-with-msi) SparkVersion: 'azure-cosmos-spark_4-0_2-13' - ClusterName: 'oltp-ci-spark40-2workers-ds3v2-scala_2.13' - AvoidDBFS: true - JavaVersion: '1.21' - JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-token) - JarStorageAccountName: 'oltpsparkcijarstore1225' - JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' - - template: /sdk/cosmos/spark.databricks.yml - parameters: - CosmosEndpointMsi: $(spark-databricks-cosmos-endpoint-msi) - CosmosEndpoint: $(spark-databricks-cosmos-endpoint) - CosmosKey: $(spark-databricks-cosmos-key) - DatabricksEndpoint: $(spark-databricks-endpoint-with-msi) - SubscriptionId: '8fba6d4f-7c37-4d13-9063-fd58ad2b86e2' - TenantId: '72f988bf-86f1-41af-91ab-2d7cd011db47' - ResourceGroupName: 'oltp-spark-ci' - ClientId: $(spark-databricks-cosmos-spn-clientId) - ClientSecret: $(spark-databricks-cosmos-spn-clientSecret) - CosmosContainerName: 'sampleContainer9' - CosmosDatabaseName: 'sampleDB9' - DatabricksToken: $(spark-databricks-token-with-msi) - SparkVersion: 'azure-cosmos-spark_4-0_2-13' - ClusterName: 'oltp-ci-spark40-2workers-ds3v2-scala_2.13' + ClusterName: 'oltp-ci-spark40-2workers-ds3v2-17.3' AvoidDBFS: true JavaVersion: '1.21' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) From 080aee286cb154b97d2cd4bee8754d224706916c Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 11:56:40 +0000 Subject: [PATCH 06/42] Fixing build issues --- .vscode/cspell.json | 1 + eng/.docsettings.yml | 1 + eng/pipelines/aggregate-reports.yml | 2 +- eng/versioning/version_client.txt | 3 ++- 4 files changed, 5 insertions(+), 2 deletions(-) diff --git a/.vscode/cspell.json b/.vscode/cspell.json index 00396b171619..d00d93ad9a92 100644 --- a/.vscode/cspell.json +++ b/.vscode/cspell.json @@ -106,6 +106,7 @@ "sdk/cosmos/azure-cosmos-spark_3-5/**", "sdk/cosmos/azure-cosmos-spark_3-5_2-12/**", "sdk/cosmos/azure-cosmos-spark_3-5_2-13/**", + "sdk/cosmos/azure-cosmos-spark_4-0_2-13/**", "sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/**", "sdk/cosmos/fabric-cosmos-spark-auth_3/**", "sdk/cosmos/azure-cosmos-encryption/**", diff --git a/eng/.docsettings.yml b/eng/.docsettings.yml index e68c43e623e6..d4ee0c5850f6 100644 --- a/eng/.docsettings.yml +++ b/eng/.docsettings.yml @@ -79,6 +79,7 @@ known_content_issues: - ['sdk/cosmos/azure-cosmos-spark_3-4_2-12/README.md', '#3113'] - ['sdk/cosmos/azure-cosmos-spark_3-5_2-12/README.md', '#3113'] - ['sdk/cosmos/azure-cosmos-spark_3-5_2-13/README.md', '#3113'] + - ['sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md', '#3113'] - ['sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/README.md', '#3113'] - ['sdk/cosmos/fabric-cosmos-spark-auth_3/README.md', '#3113'] - ['sdk/cosmos/azure-cosmos-spark_3_2-12/dev/README.md', '#3113'] diff --git a/eng/pipelines/aggregate-reports.yml b/eng/pipelines/aggregate-reports.yml index b59dafd62459..502a159ac8e9 100644 --- a/eng/pipelines/aggregate-reports.yml +++ b/eng/pipelines/aggregate-reports.yml @@ -48,7 +48,7 @@ extends: displayName: 'Build all libraries that support Java $(JavaBuildVersion)' inputs: mavenPomFile: pom.xml - options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' + options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' mavenOptions: '$(MemoryOptions) $(LoggingOptions)' javaHomeOption: 'JDKVersion' jdkVersionOption: $(JavaBuildVersion) diff --git a/eng/versioning/version_client.txt b/eng/versioning/version_client.txt index 11249debac25..d63251dff7dd 100644 --- a/eng/versioning/version_client.txt +++ b/eng/versioning/version_client.txt @@ -115,7 +115,8 @@ com.azure:azure-cosmos-test;1.0.0-beta.16;1.0.0-beta.17 com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12;4.42.0;4.43.0-beta.1 com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12;4.42.0;4.43.0-beta.1 com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12;4.42.0;4.43.0-beta.1 -com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-13;4.42.0;4.43.0-beta.1 +com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-13;4.43.0-beta.1;4.43.0-beta.1 +com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13;4.43.0-beta.1;4.43.0-beta.1 com.azure.cosmos.spark:fabric-cosmos-spark-auth_3;1.1.0;1.2.0-beta.1 com.azure:azure-cosmos-tests;1.0.0-beta.1;1.0.0-beta.1 com.azure:azure-data-appconfiguration;1.8.5;1.9.0 From 2e2cd4d2fe8ba4a9f0743fcd75965e410aac8bd9 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 14:46:10 +0000 Subject: [PATCH 07/42] Fixing build issues --- eng/versioning/external_dependencies.txt | 1 + .../azure-cosmos-spark_4-0_2-13/README.md | 75 +++++++++++-------- 2 files changed, 43 insertions(+), 33 deletions(-) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 59d308115060..cb40a67e1eff 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -20,6 +20,7 @@ com.fasterxml.jackson.datatype:jackson-datatype-jsr310;2.18.4 com.fasterxml.jackson.module:jackson-module-afterburner;2.18.4 com.fasterxml.jackson.module:jackson-module-blackbird;2.18.4 com.fasterxml.jackson.module:jackson-module-scala_2.12;2.18.4 +com.fasterxml.jackson.module:jackson-module-scala_2.13;2.18.4 com.github.spotbugs:spotbugs;4.8.3 com.github.spotbugs:spotbugs-annotations;4.8.3 com.github.spotbugs:spotbugs-maven-plugin;4.8.3.1 diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md index 1ed8a8186288..effe3b4317b5 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/README.md @@ -15,14 +15,6 @@ https://github.com/Azure/azure-sdk-for-java/issues/new - [Catalog API](https://aka.ms/azure-cosmos-spark-3-catalog-api) - [Configuration Parameter Reference](https://aka.ms/azure-cosmos-spark-3-config) -[//]: # (//TODO: add more sections) -[//]: # (//TODO: Enable Client Logging) -[//]: # (//TODO: Examples) -[//]: # (//TODO: Next steps) -[//]: # (//TODO: Key concepts) -[//]: # (//TODO: Azure Cosmos DB Partition) -[//]: # (//TODO: Troubleshooting) - ### Version Compatibility #### azure-cosmos-spark_4-0_2-13 @@ -33,39 +25,56 @@ https://github.com/Azure/azure-sdk-for-java/issues/new Note: Spark 4.0 requires Scala 2.13 and Java 17 or higher. When using the Scala API, it is necessary for applications to use Scala 2.13 that Spark 4.0 was compiled for. -### Examples -For examples please review our samples repository on [GitHub](https://github.com/Azure-Samples/azure-cosmos-spark-samples). +### Download + +You can use the maven coordinate of the jar to auto install the Spark Connector to your Databricks Runtime from Maven: +`com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13:4.43.0` + +You can also integrate against Cosmos DB Spark Connector in your SBT project: +```scala +libraryDependencies += "com.azure.cosmos.spark" % "azure-cosmos-spark_4-0_2-13" % "4.43.0" +``` -### Getting Help -Please create [GitHub issues](https://github.com/Azure/azure-sdk-for-java/issues/new) for any questions and issues. +Cosmos DB Spark Connector is available on [Maven Central Repo](https://central.sonatype.com/search?namespace=com.azure.cosmos.spark). -### Troubleshooting +#### General -See the [Troubleshooting Guide](docs/troubleshooting.md) for detailed troubleshooting tips. +If you encounter any bug, please file an issue [here](https://github.com/Azure/azure-sdk-for-java/issues/new). -### Release Notes -See [CHANGELOG.md](CHANGELOG.md) for detailed release information. +To suggest a new feature or changes that could be made, file an issue the same way you would for a bug. + +### License +This project is under MIT license and uses and repackages other third party libraries as an uber jar. +See [NOTICE.txt](https://github.com/Azure/azure-sdk-for-java/blob/main/NOTICE.txt). ### Contributing -This project welcomes contributions and suggestions. Most contributions require you to agree to a -Contributor License Agreement (CLA) declaring that you have the right to, and actually do, grant us -the rights to use your contribution. For details, visit https://cla.microsoft.com. -When you submit a pull request, a CLA-bot will automatically determine whether you need to provide -a CLA and decorate the PR appropriately (e.g., label, comment). Simply follow the instructions -provided by the bot. You will only need to do this once across all repos using our CLA. +This project welcomes contributions and suggestions. Most contributions require you to agree to a +[Contributor License Agreement (CLA)][cla] declaring that you have the right to, and actually do, grant us the rights +to use your contribution. -This project has adopted the [Microsoft Open Source Code of Conduct](https://opensource.microsoft.com/codeofconduct/). -For more information see the [Code of Conduct FAQ](https://opensource.microsoft.com/codeofconduct/faq/) or -contact [opencode@microsoft.com](mailto:opencode@microsoft.com) with any additional questions or comments. +When you submit a pull request, a CLA-bot will automatically determine whether you need to provide a CLA and decorate +the PR appropriately (e.g., label, comment). Simply follow the instructions provided by the bot. You will only need to +do this once across all repos using our CLA. -Please refer to our [Contribution guidelines](./CONTRIBUTING.md). +This project has adopted the [Microsoft Open Source Code of Conduct][coc]. For more information see the [Code of Conduct FAQ][coc_faq] +or contact [opencode@microsoft.com][coc_contact] with any additional questions or comments. - -[sql_api_query]: https://docs.microsoft.com/azure/cosmos-db/sql-api-sql-query -[cosmos_introduction]: https://docs.microsoft.com/azure/cosmos-db/ -[cosmos_docs]: https://docs.microsoft.com/azure/cosmos-db/introduction -[jdk]: https://docs.microsoft.com/java/azure/jdk/?view=azure-java-stable + +[source_code]: src +[cosmos_introduction]: https://learn.microsoft.com/azure/cosmos-db/ +[cosmos_docs]: https://learn.microsoft.com/azure/cosmos-db/introduction +[jdk]: https://learn.microsoft.com/java/azure/jdk/?view=azure-java-stable [maven]: https://maven.apache.org/ - -![Impressions](https://azure-sdk-impressions.azurewebsites.net/api/impressions/azure-sdk-for-java%2Fsdk%2Fcosmos%2Fazure-cosmos-spark_4-0_2-13%2FREADME.png) +[cla]: https://cla.microsoft.com +[coc]: https://opensource.microsoft.com/codeofconduct/ +[coc_faq]: https://opensource.microsoft.com/codeofconduct/faq/ +[coc_contact]: mailto:opencode@microsoft.com +[azure_subscription]: https://azure.microsoft.com/free/ +[samples]: https://github.com/Azure/azure-sdk-for-java/tree/main/sdk/spring/azure-spring-data-cosmos/src/samples/java/com/azure/spring/data/cosmos +[sql_api_query]: https://learn.microsoft.com/azure/cosmos-db/sql-api-sql-query +[local_emulator]: https://learn.microsoft.com/azure/cosmos-db/local-emulator +[local_emulator_export_ssl_certificates]: https://learn.microsoft.com/azure/cosmos-db/local-emulator-export-ssl-certificates +[azure_cosmos_db_partition]: https://learn.microsoft.com/azure/cosmos-db/partition-data +[sql_queries_in_cosmos]: https://learn.microsoft.com/azure/cosmos-db/tutorial-query-sql-api +[sql_queries_getting_started]: https://learn.microsoft.com/azure/cosmos-db/sql-query-getting-started From 000c114193d30891f3e0bf08cf6079035206791d Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 15:41:27 +0000 Subject: [PATCH 08/42] Fixing dependencies --- eng/versioning/external_dependencies.txt | 2 -- .../azure-cosmos-spark-account-data-resolver-sample/pom.xml | 6 ++---- sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml | 6 ++---- 3 files changed, 4 insertions(+), 10 deletions(-) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index cb40a67e1eff..dba54a86599e 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -240,8 +240,6 @@ cosmos_io.dropwizard.metrics:metrics-graphite;4.1.0 cosmos_io.dropwizard.metrics:metrics-jvm;4.1.0 cosmos_org.mpierce.metrics.reservoir:hdrhistogram-metrics-reservoir;1.1.0 cosmos_org.hdrhistogram:HdrHistogram;2.1.12 -cosmos_com.fasterxml.jackson.core:jackson-databind;2.15.2 -cosmos_com.fasterxml.jackson.module:jackson-module-scala_2.12;2.15.2 cosmos_com.microsoft.azure.synapse:synapseutils_2.12;1.5.4 ## Cosmos Spark connector under sdk\cosmos\azure-cosmos-spark_3-_2-12\pom.xml diff --git a/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml b/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml index 1e22273a8f94..a932b3fde0c1 100644 --- a/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml @@ -75,12 +75,12 @@ com.fasterxml.jackson.core jackson-databind - 2.15.2 + 2.18.4 com.fasterxml.jackson.module jackson-module-scala_2.12 - 2.15.2 + 2.18.4 org.scala-lang @@ -289,8 +289,6 @@ com.fasterxml.jackson.datatype:jackson-datatype-jsr310:[2.18.4] com.fasterxml.jackson.core:jackson-databind:[2.18.4] com.fasterxml.jackson.module:jackson-module-scala_2.12:[2.18.4] - com.fasterxml.jackson.core:jackson-databind:[2.15.2] - com.fasterxml.jackson.module:jackson-module-scala_2.12:[2.15.2] com.globalmentor:hadoop-bare-naked-local-fs:[0.1.0] com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12:[4.43.0-beta.1] diff --git a/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml b/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml index 45524afbe974..198a80f41555 100644 --- a/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml +++ b/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml @@ -66,12 +66,12 @@ com.fasterxml.jackson.core jackson-databind - 2.15.2 + 2.18.4 com.fasterxml.jackson.module jackson-module-scala_2.12 - 2.15.2 + 2.18.4 org.scala-lang @@ -183,8 +183,6 @@ com.fasterxml.jackson.datatype:jackson-datatype-jsr310:[2.18.4] com.fasterxml.jackson.core:jackson-databind:[2.18.4] com.fasterxml.jackson.module:jackson-module-scala_2.12:[2.18.4] - com.fasterxml.jackson.core:jackson-databind:[2.15.2] - com.fasterxml.jackson.module:jackson-module-scala_2.12:[2.15.2] com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12:[4.43.0-beta.1] com.microsoft.azure.synapse:synapseutils_2.12:[1.5.4] From 350919c39513696e6bdc8ac08e725068950d361d Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 15:51:24 +0000 Subject: [PATCH 09/42] Fixing dependency issues --- .../azure-cosmos-spark-account-data-resolver-sample/pom.xml | 2 +- sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml b/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml index a932b3fde0c1..c85788d55947 100644 --- a/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark-account-data-resolver-sample/pom.xml @@ -80,7 +80,7 @@ com.fasterxml.jackson.module jackson-module-scala_2.12 - 2.18.4 + 2.18.4 org.scala-lang diff --git a/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml b/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml index 198a80f41555..9205a8211e62 100644 --- a/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml +++ b/sdk/cosmos/fabric-cosmos-spark-auth_3/pom.xml @@ -71,7 +71,7 @@ com.fasterxml.jackson.module jackson-module-scala_2.12 - 2.18.4 + 2.18.4 org.scala-lang From da17e72b3823ec44fbb0f1ae2eb9cd51f1c0e512 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 17:44:27 +0000 Subject: [PATCH 10/42] Update databricks-jar-install.sh --- .../test-databricks/databricks-jar-install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/cosmos/azure-cosmos-spark_3/test-databricks/databricks-jar-install.sh b/sdk/cosmos/azure-cosmos-spark_3/test-databricks/databricks-jar-install.sh index d0a16daa4f67..c1a1b2389a73 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/test-databricks/databricks-jar-install.sh +++ b/sdk/cosmos/azure-cosmos-spark_3/test-databricks/databricks-jar-install.sh @@ -39,7 +39,7 @@ JAR_CHECK_SUM="$(sha256sum -- "$JARPATH/$JARFILE")" || { exit 1 } JAR_CHECK_SUM="${JAR_CHECK_SUM%% *}" -echo "CHECKSUM of the jar (used to ensure there are no concurrent live tests interfering) - $JAR_CHECK_SUM" +echo "CHECKSUM of the jar $JARPATH/$JARFILE (used to ensure there are no concurrent live tests interfering) - $JAR_CHECK_SUM" echo "##vso[task.setvariable variable=JarCheckSum]$JAR_CHECK_SUM" echo "CLUSTER_NAME: $CLUSTER_NAME" From 425380c09f9a860c70d6faff4ba960ad421576f9 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 18:20:35 +0000 Subject: [PATCH 11/42] Update spark.yml --- sdk/cosmos/spark.yml | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index e2ed7791ced5..5149e2def1de 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -6,6 +6,12 @@ variables: - template: /eng/pipelines/templates/variables/image.yml - name: AdditionalArgs value: '' + - name: JarReadOnlySasUriScala212 + value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[0] ] + - name: JarReadOnlySasUriSpark35Scala213 + value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[1] ] + - name: JarReadOnlySasUriSpark40Scala213 + value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[2] ] stages: - template: /sdk/cosmos/spark.databricks.yml @@ -25,7 +31,7 @@ stages: SparkVersion: 'azure-cosmos-spark_3-3_2-12' ClusterName: 'oltp-ci-spark33-2workers-ds3v2' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUri: $(JarReadOnlySasUriScala212) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -45,7 +51,7 @@ stages: SparkVersion: 'azure-cosmos-spark_3-4_2-12' ClusterName: 'oltp-ci-spark34-2workers-ds3v2' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUri: $(JarReadOnlySasUriScala212) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -66,7 +72,7 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2' AvoidDBFS: false JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUri: $(JarReadOnlySasUriScala212) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -87,7 +93,7 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4' AvoidDBFS: true JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUri: $(JarReadOnlySasUriScala212) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -108,7 +114,7 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4-scala_2.13' AvoidDBFS: true JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-token) + JarReadOnlySasUri: $(JarReadOnlySasUriSpark35Scala213) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-13-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -130,6 +136,6 @@ stages: AvoidDBFS: true JavaVersion: '1.21' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(spark-databricks-token) + JarReadOnlySasUri: $(JarReadOnlySasUriSpark40Scala213) JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' From 5222aceda18b4972ed9355c7cd7df109a3df7330 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 18:58:47 +0000 Subject: [PATCH 12/42] Update spark.databricks.yml --- sdk/cosmos/spark.databricks.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/sdk/cosmos/spark.databricks.yml b/sdk/cosmos/spark.databricks.yml index 6b62dfbefe06..a75abf352ea6 100644 --- a/sdk/cosmos/spark.databricks.yml +++ b/sdk/cosmos/spark.databricks.yml @@ -112,6 +112,7 @@ stages: if [[ "${AVOID_DBFS,,}" == "true" ]]; then TMP_FILE="/tmp/${JAR_NAME}.$$" echo "[init] Downloading $JAR_NAME to validate checksum..." + echo "[URI] $JAR_URL" # Download with retries; fail non-zero on HTTP errors if command -v curl >/dev/null 2>&1; then curl -fL --retry 8 --retry-connrefused --retry-delay 2 --max-time 600 \ From 61a963c45896b741fe0b15c4dd4a00bc7a493a91 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 20:03:19 +0000 Subject: [PATCH 13/42] Update aggregate-reports.yml --- eng/pipelines/aggregate-reports.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eng/pipelines/aggregate-reports.yml b/eng/pipelines/aggregate-reports.yml index 502a159ac8e9..d7ee07ac7976 100644 --- a/eng/pipelines/aggregate-reports.yml +++ b/eng/pipelines/aggregate-reports.yml @@ -48,7 +48,7 @@ extends: displayName: 'Build all libraries that support Java $(JavaBuildVersion)' inputs: mavenPomFile: pom.xml - options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' + options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' mavenOptions: '$(MemoryOptions) $(LoggingOptions)' javaHomeOption: 'JDKVersion' jdkVersionOption: $(JavaBuildVersion) From 73f24d3f72ee13ee32613d74f01ad69ac968cb8a Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 20:13:34 +0000 Subject: [PATCH 14/42] Create azure-cosmos-spark.properties --- .../src/main/resources/azure-cosmos-spark.properties | 2 ++ 1 file changed, 2 insertions(+) create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/resources/azure-cosmos-spark.properties diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/resources/azure-cosmos-spark.properties b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/resources/azure-cosmos-spark.properties new file mode 100644 index 000000000000..ca812989b4f2 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/main/resources/azure-cosmos-spark.properties @@ -0,0 +1,2 @@ +name=${project.artifactId} +version=${project.version} From f43acac710449c6403c6fa349e365f4b02272754 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 21:03:09 +0000 Subject: [PATCH 15/42] Fixed handling of SAS Uris --- sdk/cosmos/spark.databricks.yml | 19 ++++++++++++++++--- sdk/cosmos/spark.yml | 24 ++++++++++++------------ 2 files changed, 28 insertions(+), 15 deletions(-) diff --git a/sdk/cosmos/spark.databricks.yml b/sdk/cosmos/spark.databricks.yml index a75abf352ea6..5ff8cb1b6738 100644 --- a/sdk/cosmos/spark.databricks.yml +++ b/sdk/cosmos/spark.databricks.yml @@ -34,6 +34,8 @@ parameters: type: string - name: JarReadOnlySasUri type: string + - name: JarReadOnlySasUriIndex + type: int - name: JarStorageAccountName type: string - name: JarName @@ -111,14 +113,24 @@ stages: script: >- if [[ "${AVOID_DBFS,,}" == "true" ]]; then TMP_FILE="/tmp/${JAR_NAME}.$$" + + IFS='|' read -r -a JAR_URL_ARRAY <<< "$JAR_URL" + + if [[ -z "${JAR_URL_ARRAY[$JAR_URL_INDEX]:-}" ]]; then + echo "[error] Invalid JAR_URL_INDEX=$JAR_URL_INDEX" + exit 1 + fi + + JAR_URL_SELECTED="$(echo "${JAR_URL_ARRAY[$JAR_URL_INDEX]}" | xargs)" + echo "[init] Downloading $JAR_NAME to validate checksum..." - echo "[URI] $JAR_URL" + echo "[URI] $JAR_URL_SELECTED" # Download with retries; fail non-zero on HTTP errors if command -v curl >/dev/null 2>&1; then curl -fL --retry 8 --retry-connrefused --retry-delay 2 --max-time 600 \ - "$JAR_URL" -o "$TMP_FILE" + "$JAR_URL_SELECTED" -o "$TMP_FILE" elif command -v wget >/dev/null 2>&1; then - wget --tries=10 --timeout=60 -O "$TMP_FILE" "$JAR_URL" + wget --tries=10 --timeout=60 -O "$TMP_FILE" "$JAR_URL_SELECTED" else echo "[init][error] Neither curl nor wget is available." >&2 exit 1 @@ -143,6 +155,7 @@ stages: fi env: JAR_URL: '${{ parameters.JarReadOnlySasUri }}' + JAR_URL_INDEX: '${{ parameters.JarReadOnlySasUriIndex }}' JAR_NAME: '${{ parameters.SparkVersion }}-latest-ci-candidate.jar' JAR_CHECK_SUM: $(JarCheckSum) AVOID_DBFS: ${{ parameters.AvoidDBFS }} diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index 5149e2def1de..5caa6a0c911f 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -6,12 +6,6 @@ variables: - template: /eng/pipelines/templates/variables/image.yml - name: AdditionalArgs value: '' - - name: JarReadOnlySasUriScala212 - value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[0] ] - - name: JarReadOnlySasUriSpark35Scala213 - value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[1] ] - - name: JarReadOnlySasUriSpark40Scala213 - value: $[ split(variables['spark-databricks-cosmos-spn-clientCertBase64'], '|')[2] ] stages: - template: /sdk/cosmos/spark.databricks.yml @@ -31,7 +25,8 @@ stages: SparkVersion: 'azure-cosmos-spark_3-3_2-12' ClusterName: 'oltp-ci-spark33-2workers-ds3v2' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriScala212) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 0 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -51,7 +46,8 @@ stages: SparkVersion: 'azure-cosmos-spark_3-4_2-12' ClusterName: 'oltp-ci-spark34-2workers-ds3v2' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriScala212) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 0 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -72,7 +68,8 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2' AvoidDBFS: false JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriScala212) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 0 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -93,7 +90,8 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4' AvoidDBFS: true JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriScala212) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 0 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-12-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -114,7 +112,8 @@ stages: ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4-scala_2.13' AvoidDBFS: true JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriSpark35Scala213) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 1 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_3-5_2-13-latest-ci-candidate.jar' - template: /sdk/cosmos/spark.databricks.yml @@ -136,6 +135,7 @@ stages: AvoidDBFS: true JavaVersion: '1.21' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) - JarReadOnlySasUri: $(JarReadOnlySasUriSpark40Scala213) + JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) + JarReadOnlySasUriIndex: 2 JarStorageAccountName: 'oltpsparkcijarstore1225' JarName: 'azure-cosmos-spark_4-0_2-13-latest-ci-candidate.jar' From 92651fc5755d62344c9ebe819f3a9f25e295c9d7 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 21:31:52 +0000 Subject: [PATCH 16/42] Update spark.databricks.yml --- sdk/cosmos/spark.databricks.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/cosmos/spark.databricks.yml b/sdk/cosmos/spark.databricks.yml index 5ff8cb1b6738..62a610c3e3fc 100644 --- a/sdk/cosmos/spark.databricks.yml +++ b/sdk/cosmos/spark.databricks.yml @@ -35,7 +35,7 @@ parameters: - name: JarReadOnlySasUri type: string - name: JarReadOnlySasUriIndex - type: int + type: number - name: JarStorageAccountName type: string - name: JarName From 74fd3014cd80fd8ebb0c854060dad0bd324f4611 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 22:56:05 +0000 Subject: [PATCH 17/42] Fixing test failures --- .../spark/ChangeFeedMetricsListenerSpec.scala | 0 .../spark/ChangeFeedMetricsListenerSpec.scala | 138 ++++++++++++++++++ .../spark/ChangeFeedMetricsListenerSpec.scala | 138 ++++++++++++++++++ .../spark/ChangeFeedMetricsListenerSpec.scala | 138 ++++++++++++++++++ sdk/cosmos/spark.yml | 5 + 5 files changed, 419 insertions(+) rename sdk/cosmos/{azure-cosmos-spark_3 => azure-cosmos-spark_3-3_2-12}/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala (100%) create mode 100644 sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala create mode 100644 sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala diff --git a/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala similarity index 100% rename from sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala rename to sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala new file mode 100644 index 000000000000..70d0352369a6 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.changeFeedMetrics.{ChangeFeedMetricsListener, ChangeFeedMetricsTracker} +import com.azure.cosmos.implementation.guava25.collect.{HashBiMap, Maps} +import org.apache.spark.Success +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{SparkListenerTaskEnd, TaskInfo} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.mockito.ArgumentMatchers +import org.mockito.Mockito.{mock, when} + +import java.lang.reflect.Field +import java.util.concurrent.ConcurrentHashMap + +class ChangeFeedMetricsListenerSpec extends UnitSpec { + "ChangeFeedMetricsListener" should "be able to capture changeFeed performance metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("index", 1), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // verify that metrics will be properly tracked + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap.size() shouldBe 1 + partitionMetricsMap.containsKey(NormalizedRange("0", "FF")) shouldBe true + partitionMetricsMap.get(NormalizedRange("0", "FF")).getWeightedChangeFeedItemsPerLsn.get shouldBe 1 + } + + it should "ignore metrics for unknown partition index" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("index", 10), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } + + it should "ignore unrelated metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + "unknownMetrics" -> new SQLMetric("index", 10) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala new file mode 100644 index 000000000000..70d0352369a6 --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.changeFeedMetrics.{ChangeFeedMetricsListener, ChangeFeedMetricsTracker} +import com.azure.cosmos.implementation.guava25.collect.{HashBiMap, Maps} +import org.apache.spark.Success +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{SparkListenerTaskEnd, TaskInfo} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.mockito.ArgumentMatchers +import org.mockito.Mockito.{mock, when} + +import java.lang.reflect.Field +import java.util.concurrent.ConcurrentHashMap + +class ChangeFeedMetricsListenerSpec extends UnitSpec { + "ChangeFeedMetricsListener" should "be able to capture changeFeed performance metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("index", 1), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // verify that metrics will be properly tracked + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap.size() shouldBe 1 + partitionMetricsMap.containsKey(NormalizedRange("0", "FF")) shouldBe true + partitionMetricsMap.get(NormalizedRange("0", "FF")).getWeightedChangeFeedItemsPerLsn.get shouldBe 1 + } + + it should "ignore metrics for unknown partition index" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("index", 10), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } + + it should "ignore unrelated metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + "unknownMetrics" -> new SQLMetric("index", 10) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } +} diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala new file mode 100644 index 000000000000..6d203ed81b9a --- /dev/null +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -0,0 +1,138 @@ +// Copyright (c) Microsoft Corporation. All rights reserved. +// Licensed under the MIT License. + +package com.azure.cosmos.spark + +import com.azure.cosmos.changeFeedMetrics.{ChangeFeedMetricsListener, ChangeFeedMetricsTracker} +import com.azure.cosmos.implementation.guava25.collect.{HashBiMap, Maps} +import org.apache.spark.Success +import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.scheduler.{SparkListenerTaskEnd, TaskInfo} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.mockito.ArgumentMatchers +import org.mockito.Mockito.{mock, when} + +import java.lang.reflect.Field +import java.util.concurrent.ConcurrentHashMap + +class ChangeFeedMetricsListenerSpec extends UnitSpec { + "ChangeFeedMetricsListener" should "be able to capture changeFeed performance metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("sum", "index", 1), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("sum", "lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("sum", "items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // verify that metrics will be properly tracked + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap.size() shouldBe 1 + partitionMetricsMap.containsKey(NormalizedRange("0", "FF")) shouldBe true + partitionMetricsMap.get(NormalizedRange("0", "FF")).getWeightedChangeFeedItemsPerLsn.get shouldBe 1 + } + + it should "ignore metrics for unknown partition index" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("sum", "index", 10), + CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("sum", "lsn", 100), + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("sum", "items", 100) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } + + it should "ignore unrelated metrics" in { + val taskEnd = SparkListenerTaskEnd( + stageId = 1, + stageAttemptId = 0, + taskType = "ResultTask", + reason = Success, + taskInfo = mock(classOf[TaskInfo]), + taskExecutorMetrics = mock(classOf[ExecutorMetrics]), + taskMetrics = mock(classOf[TaskMetrics]) + ) + + val metrics = Map[String, SQLMetric]( + "unknownMetrics" -> new SQLMetric("sum", "index", 10) + ) + + // create sparkInternalsBridge mock + val sparkInternalsBridge = mock(classOf[SparkInternalsBridge]) + when(sparkInternalsBridge.getInternalCustomTaskMetricsAsSQLMetric( + ArgumentMatchers.any[Set[String]], + ArgumentMatchers.any[TaskMetrics] + )).thenReturn(metrics) + + val partitionIndexMap = Maps.synchronizedBiMap(HashBiMap.create[NormalizedRange, Long]()) + partitionIndexMap.put(NormalizedRange("0", "FF"), 1) + + val partitionMetricsMap = new ConcurrentHashMap[NormalizedRange, ChangeFeedMetricsTracker]() + val changeFeedMetricsListener = new ChangeFeedMetricsListener(partitionIndexMap, partitionMetricsMap) + + // set the internal sparkInternalsBridgeField + val sparkInternalsBridgeField: Field = classOf[ChangeFeedMetricsListener].getDeclaredField("sparkInternalsBridge") + sparkInternalsBridgeField.setAccessible(true) + sparkInternalsBridgeField.set(changeFeedMetricsListener, sparkInternalsBridge) + + // because partition index 10 does not exist in the partitionIndexMap, it will be ignored + changeFeedMetricsListener.onTaskEnd(taskEnd) + partitionMetricsMap shouldBe empty + } +} diff --git a/sdk/cosmos/spark.yml b/sdk/cosmos/spark.yml index 5caa6a0c911f..a8820e948b58 100644 --- a/sdk/cosmos/spark.yml +++ b/sdk/cosmos/spark.yml @@ -24,6 +24,7 @@ stages: DatabricksToken: $(spark-databricks-token-with-msi) SparkVersion: 'azure-cosmos-spark_3-3_2-12' ClusterName: 'oltp-ci-spark33-2workers-ds3v2' + JavaVersion: '1.11' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) JarReadOnlySasUriIndex: 0 @@ -45,6 +46,7 @@ stages: DatabricksToken: $(spark-databricks-token-with-msi) SparkVersion: 'azure-cosmos-spark_3-4_2-12' ClusterName: 'oltp-ci-spark34-2workers-ds3v2' + JavaVersion: '1.11' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) JarReadOnlySasUriIndex: 0 @@ -67,6 +69,7 @@ stages: SparkVersion: 'azure-cosmos-spark_3-5_2-12' ClusterName: 'oltp-ci-spark35-2workers-ds3v2' AvoidDBFS: false + JavaVersion: '1.11' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) JarReadOnlySasUriIndex: 0 @@ -89,6 +92,7 @@ stages: SparkVersion: 'azure-cosmos-spark_3-5_2-12' ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4' AvoidDBFS: true + JavaVersion: '1.11' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) JarReadOnlySasUriIndex: 0 @@ -111,6 +115,7 @@ stages: SparkVersion: 'azure-cosmos-spark_3-5_2-13' ClusterName: 'oltp-ci-spark35-2workers-ds3v2-16.4-scala_2.13' AvoidDBFS: true + JavaVersion: '1.11' JarStorageAccountKey: $(spark-databricks-cosmos-spn-clientIdCert) JarReadOnlySasUri: $(spark-databricks-cosmos-spn-clientCertBase64) JarReadOnlySasUriIndex: 1 From 8fb735dc39c77a542a7c71e5f65a9b8f8e6037d6 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Thu, 15 Jan 2026 23:19:06 +0000 Subject: [PATCH 18/42] Fixing build issues --- .../com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala | 2 ++ .../com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala | 2 ++ .../com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala | 2 ++ .../com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala | 2 ++ 4 files changed, 8 insertions(+) diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala index 70d0352369a6..8c9ec08a23de 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -1,5 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. +// scalastyle:off magic.number +// scalastyle:off multiple.string.literals package com.azure.cosmos.spark diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala index 70d0352369a6..8c9ec08a23de 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -1,5 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. +// scalastyle:off magic.number +// scalastyle:off multiple.string.literals package com.azure.cosmos.spark diff --git a/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala index 70d0352369a6..8c9ec08a23de 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala +++ b/sdk/cosmos/azure-cosmos-spark_3-5/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -1,5 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. +// scalastyle:off magic.number +// scalastyle:off multiple.string.literals package com.azure.cosmos.spark diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala index 6d203ed81b9a..a41251428f78 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala @@ -1,5 +1,7 @@ // Copyright (c) Microsoft Corporation. All rights reserved. // Licensed under the MIT License. +// scalastyle:off magic.number +// scalastyle:off multiple.string.literals package com.azure.cosmos.spark From 5d6c0c9269c192d80bad408c7563e5809cf38fc1 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Fri, 16 Jan 2026 13:40:05 +0000 Subject: [PATCH 19/42] Fixing test failure --- ...a => ChangeFeedMetricsListenerITest.scala} | 35 ++++++++++++++----- 1 file changed, 26 insertions(+), 9 deletions(-) rename sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/{ChangeFeedMetricsListenerSpec.scala => ChangeFeedMetricsListenerITest.scala} (80%) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerITest.scala similarity index 80% rename from sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala rename to sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerITest.scala index a41251428f78..6b9de815ea9c 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerSpec.scala +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/ChangeFeedMetricsListenerITest.scala @@ -10,14 +10,14 @@ import com.azure.cosmos.implementation.guava25.collect.{HashBiMap, Maps} import org.apache.spark.Success import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.{SparkListenerTaskEnd, TaskInfo} -import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.mockito.ArgumentMatchers import org.mockito.Mockito.{mock, when} import java.lang.reflect.Field import java.util.concurrent.ConcurrentHashMap -class ChangeFeedMetricsListenerSpec extends UnitSpec { +class ChangeFeedMetricsListenerITest extends IntegrationSpec with SparkWithJustDropwizardAndNoSlf4jMetrics { "ChangeFeedMetricsListener" should "be able to capture changeFeed performance metrics" in { val taskEnd = SparkListenerTaskEnd( stageId = 1, @@ -29,10 +29,17 @@ class ChangeFeedMetricsListenerSpec extends UnitSpec { taskMetrics = mock(classOf[TaskMetrics]) ) + val indexMetric = SQLMetrics.createMetric(spark.sparkContext, "index") + indexMetric.set(1) + val lsnMetric = SQLMetrics.createMetric(spark.sparkContext, "lsn") + lsnMetric.set(100) + val itemsMetric = SQLMetrics.createMetric(spark.sparkContext, "items") + itemsMetric.set(100) + val metrics = Map[String, SQLMetric]( - CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("sum", "index", 1), - CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("sum", "lsn", 100), - CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("sum", "items", 100) + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> indexMetric, + CosmosConstants.MetricNames.ChangeFeedLsnRange -> lsnMetric, + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> itemsMetric ) // create sparkInternalsBridge mock @@ -71,10 +78,17 @@ class ChangeFeedMetricsListenerSpec extends UnitSpec { taskMetrics = mock(classOf[TaskMetrics]) ) + val indexMetric2 = SQLMetrics.createMetric(spark.sparkContext, "index") + indexMetric2.set(10) + val lsnMetric2 = SQLMetrics.createMetric(spark.sparkContext, "lsn") + lsnMetric2.set(100) + val itemsMetric2 = SQLMetrics.createMetric(spark.sparkContext, "items") + itemsMetric2.set(100) + val metrics = Map[String, SQLMetric]( - CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> new SQLMetric("sum", "index", 10), - CosmosConstants.MetricNames.ChangeFeedLsnRange -> new SQLMetric("sum", "lsn", 100), - CosmosConstants.MetricNames.ChangeFeedItemsCnt -> new SQLMetric("sum", "items", 100) + CosmosConstants.MetricNames.ChangeFeedPartitionIndex -> indexMetric2, + CosmosConstants.MetricNames.ChangeFeedLsnRange -> lsnMetric2, + CosmosConstants.MetricNames.ChangeFeedItemsCnt -> itemsMetric2 ) // create sparkInternalsBridge mock @@ -111,8 +125,11 @@ class ChangeFeedMetricsListenerSpec extends UnitSpec { taskMetrics = mock(classOf[TaskMetrics]) ) + val unknownMetric3 = SQLMetrics.createMetric(spark.sparkContext, "unknown") + unknownMetric3.set(10) + val metrics = Map[String, SQLMetric]( - "unknownMetrics" -> new SQLMetric("sum", "index", 10) + "unknownMetrics" -> unknownMetric3 ) // create sparkInternalsBridge mock From 0c1f7d9a342a40d329b80bac6e101d0e9fbcae4e Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Fri, 16 Jan 2026 15:29:17 +0000 Subject: [PATCH 20/42] Update spark.databricks.yml --- sdk/cosmos/spark.databricks.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdk/cosmos/spark.databricks.yml b/sdk/cosmos/spark.databricks.yml index 62a610c3e3fc..2903b030f9eb 100644 --- a/sdk/cosmos/spark.databricks.yml +++ b/sdk/cosmos/spark.databricks.yml @@ -124,7 +124,7 @@ stages: JAR_URL_SELECTED="$(echo "${JAR_URL_ARRAY[$JAR_URL_INDEX]}" | xargs)" echo "[init] Downloading $JAR_NAME to validate checksum..." - echo "[URI] $JAR_URL_SELECTED" + # Download with retries; fail non-zero on HTTP errors if command -v curl >/dev/null 2>&1; then curl -fL --retry 8 --retry-connrefused --retry-delay 2 --max-time 600 \ From 0018f46d868979c8757de0129372cfa16a7b277e Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Fri, 16 Jan 2026 17:37:20 +0000 Subject: [PATCH 21/42] Debugging failing scala spark 4 unit tests when running with java 11 --- sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml | 4 +++- sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml | 4 +++- sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 4 +++- 3 files changed, 9 insertions(+), 3 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml index 5bbe86d16d50..56fa25ef44f1 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml @@ -161,6 +161,8 @@ 2.1.0 ${scalatest.argLine} + WDEF + WDEF ${project.build.directory}/surefire-reports . SparkTestSuite.txt @@ -184,7 +186,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml index c91292394edc..0100eaf574f4 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml @@ -141,6 +141,8 @@ 2.1.0 ${scalatest.argLine} + WDEF + WDEF ${project.build.directory}/surefire-reports . SparkTestSuite.txt @@ -164,7 +166,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index e2b97d65bb1c..b4b0469b8346 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -141,6 +141,8 @@ 2.1.0 ${scalatest.argLine} + WDEF + WDEF ${project.build.directory}/surefire-reports . SparkTestSuite.txt @@ -164,7 +166,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED From 9c229d3dfe9a0a200a3a7afd0d83f601568e842a Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Mon, 19 Jan 2026 11:36:53 +0000 Subject: [PATCH 22/42] Fixing unit tests --- .../spark/SparkE2EConfigResolutionITest.scala | 25 ++++++++++++++++- .../azure-cosmos-spark_4-0_2-13/pom.xml | 27 ++++++++++++++++--- 2 files changed, 48 insertions(+), 4 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/SparkE2EConfigResolutionITest.scala b/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/SparkE2EConfigResolutionITest.scala index 323932bf5477..4f045c581ba3 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/SparkE2EConfigResolutionITest.scala +++ b/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/SparkE2EConfigResolutionITest.scala @@ -87,7 +87,6 @@ class SparkE2EConfigResolutionITest extends IntegrationSpec with CosmosClient wi val cosmosEndpoint = TestConfigurations.HOST val cosmosMasterKey = TestConfigurations.MASTER_KEY - val container = cosmosClient.getDatabase(cosmosDatabase).getContainer(cosmosContainer) val objectNode = Utils.getSimpleObjectMapper.createObjectNode() objectNode.put("word", "Cobalt atomic number") @@ -125,6 +124,16 @@ class SparkE2EConfigResolutionITest extends IntegrationSpec with CosmosClient wi } it should "validate config names with 'spark.cosmos.' prefix" in { + val sparkConfig = new SparkConf() + + val spark = SparkSession.builder() + .appName("spark connector sample") + .master("local") + .config(sparkConfig) + .getOrCreate() + + LocalJavaFileSystem.applyToSparkSession(spark) + val userConfig = Map( "spark.cosmos.accountEndpoint" -> "https://boson-test.documents.azure.com:443/", "spark.cosmos.accountKey" -> "xyz", @@ -138,9 +147,21 @@ class SparkE2EConfigResolutionITest extends IntegrationSpec with CosmosClient wi case e: Exception => e.getMessage shouldEqual "The config property 'spark.cosmos.someTypo' is invalid. No config setting with this name exists." } + + spark.close() } it should "not validate config names without 'spark.cosmos.' prefix" in { + val sparkConfig = new SparkConf() + + val spark = SparkSession.builder() + .appName("spark connector sample") + .master("local") + .config(sparkConfig) + .getOrCreate() + + LocalJavaFileSystem.applyToSparkSession(spark) + val userConfig = Map( "spark.cosmos.accountEndpoint" -> "https://boson-test.documents.azure.com:443/", "spark.cosmos.accountKey" -> "xyz", @@ -148,6 +169,8 @@ class SparkE2EConfigResolutionITest extends IntegrationSpec with CosmosClient wi ) CosmosConfig.getEffectiveConfig(None, None, userConfig) + + spark.close() } //scalastyle:on magic.number diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index b4b0469b8346..b6eb7b1ef282 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -112,12 +112,11 @@ spark-e2e_4-0_2-13 + + [17,) ${basedir}/scalastyle_config.xml - - spark-e2e_4-0_2-13 - @@ -160,6 +159,28 @@ + + + spark-disable-tests-java8-11 + + (,12) + + + + + org.scalatest + scalatest-maven-plugin + 2.1.0 + + + test + none + + + + + + java9-plus From 2c14d12b31e393fedc21dc290ea93e1eb655bf05 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Mon, 19 Jan 2026 12:59:15 +0000 Subject: [PATCH 23/42] Fixing unit tests --- sdk/cosmos/azure-cosmos-spark_3/pom.xml | 68 ------------------- .../azure-cosmos-spark_4-0_2-13/pom.xml | 22 ------ 2 files changed, 90 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark_3/pom.xml b/sdk/cosmos/azure-cosmos-spark_3/pom.xml index f94cd5093e59..db68fae6da82 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3/pom.xml @@ -744,74 +744,6 @@ - - - unit - - - ${basedir}/scalastyle_config.xml - - - - - - org.apache.maven.plugins - maven-surefire-plugin - 3.5.3 - - - true - 1 - 256 - paranoid - - - **/*.* - **/*Test.* - **/*Suite.* - **/*Spec.* - - true - - - surefire.testng.verbose - 2 - - - listener - com.azure.cosmos.CosmosNettyLeakDetectorFactory - - - - - - - - org.scalatest - scalatest-maven-plugin - 2.1.0 - - - ${cosmos-spark-version} - - ${project.build.directory}/surefire-reports - . - SparkTestSuite.txt - (Spec|[^I]Test|Suite) - - - - test - - test - - - - - - - - java8 diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index b6eb7b1ef282..8946789f47c1 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -159,28 +159,6 @@ - - - spark-disable-tests-java8-11 - - (,12) - - - - - org.scalatest - scalatest-maven-plugin - 2.1.0 - - - test - none - - - - - - java9-plus From 3afb6f4738e223e6fbcbd27815c618e1d7207e32 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Mon, 19 Jan 2026 15:02:02 +0000 Subject: [PATCH 24/42] Fix unit test failures --- sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index 8946789f47c1..e7a0a6a62e9a 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -159,6 +159,16 @@ + + + spark-4-0-disable-tests-java-lt-17 + + (,17) + + + true + + java9-plus From daa3798a4b2e147e90cac9bdf453c2849cfa6d33 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Mon, 19 Jan 2026 16:23:30 +0000 Subject: [PATCH 25/42] Update pom.xml --- sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index e7a0a6a62e9a..3e3eee501c71 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -146,6 +146,8 @@ . SparkTestSuite.txt (ITest|Test|Spec|Suite) + true + once From f8353d1d94af58549c2d942b4204e5cf57baef41 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Mon, 19 Jan 2026 16:47:16 +0000 Subject: [PATCH 26/42] Update pom.xml --- sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index 3e3eee501c71..ce6b0ece032c 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -140,14 +140,17 @@ 2.1.0 ${scalatest.argLine} - WDEF - WDEF + FDEF + FDEF + + + none + ${project.build.directory}/surefire-reports . SparkTestSuite.txt (ITest|Test|Spec|Suite) true - once From bdc9d517fc22dd3c589e68d8bc6da0cd238137a8 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 10:02:53 +0000 Subject: [PATCH 27/42] Try to get better scalatest failures --- eng/pipelines/aggregate-reports.yml | 2 +- sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml | 4 ++++ sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml | 4 ++++ sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml | 8 +++++--- sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml | 8 +++++--- sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 9 +++------ 6 files changed, 22 insertions(+), 13 deletions(-) diff --git a/eng/pipelines/aggregate-reports.yml b/eng/pipelines/aggregate-reports.yml index d7ee07ac7976..371c5fa33177 100644 --- a/eng/pipelines/aggregate-reports.yml +++ b/eng/pipelines/aggregate-reports.yml @@ -48,7 +48,7 @@ extends: displayName: 'Build all libraries that support Java $(JavaBuildVersion)' inputs: mavenPomFile: pom.xml - options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' + options: '$(DefaultOptions) -T 2C -DskipTests -Dgpg.skip -Dmaven.javadoc.skip=true -Dcodesnippet.skip=true -Dcheckstyle.skip=true -Dspotbugs.skip=true -Djacoco.skip=true -Drevapi.skip=true -Dshade.skip=true -Dspotless.skip=true -pl !com.azure.cosmos.spark:azure-cosmos-spark_3-3_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-4_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-12,!com.azure.cosmos.spark:azure-cosmos-spark_3-5_2-13,!com.azure.cosmos.spark:azure-cosmos-spark_4-0_2-13,!com.azure.cosmos.spark:azure-cosmos-spark-account-data-resolver-sample,!com.azure.cosmos.kafka:azure-cosmos-kafka-connect,!com.microsoft.azure:azure-batch' mavenOptions: '$(MemoryOptions) $(LoggingOptions)' javaHomeOption: 'JDKVersion' jdkVersionOption: $(JavaBuildVersion) diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml index 5683d8b2b058..ea95cdf00179 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml @@ -148,6 +148,10 @@ scalatest-maven-plugin 2.1.0 + FDEF + FDEF + once + true ${project.build.directory}/surefire-reports . SparkTestSuite.txt diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml index 44a83888e577..25ae31a136f3 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml @@ -148,6 +148,10 @@ scalatest-maven-plugin 2.1.0 + FDEF + FDEF + once + true ${project.build.directory}/surefire-reports . SparkTestSuite.txt diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml index 56fa25ef44f1..219b0f50f262 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml @@ -161,8 +161,10 @@ 2.1.0 ${scalatest.argLine} - WDEF - WDEF + FDEF + FDEF + once + true ${project.build.directory}/surefire-reports . SparkTestSuite.txt @@ -186,7 +188,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml index 0100eaf574f4..10fd86e64bf4 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml @@ -141,8 +141,10 @@ 2.1.0 ${scalatest.argLine} - WDEF - WDEF + FDEF + FDEF + once + true ${project.build.directory}/surefire-reports . SparkTestSuite.txt @@ -166,7 +168,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index ce6b0ece032c..31a8f614bfbe 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -142,15 +142,12 @@ ${scalatest.argLine} FDEF FDEF - - - none - + once + true ${project.build.directory}/surefire-reports . SparkTestSuite.txt (ITest|Test|Spec|Suite) - true @@ -180,7 +177,7 @@ [9,) - --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED + --add-opens=java.base/java.lang=ALL-UNNAMED --add-opens=java.base/java.lang.invoke=ALL-UNNAMED --add-opens=java.base/java.lang.reflect=ALL-UNNAMED --add-opens=java.base/java.io=ALL-UNNAMED --add-opens=java.base/java.net=ALL-UNNAMED --add-opens=java.base/java.nio=ALL-UNNAMED --add-opens=java.base/java.util=ALL-UNNAMED --add-opens=java.base/java.util.concurrent=ALL-UNNAMED --add-opens=java.base/java.util.concurrent.atomic=ALL-UNNAMED --add-opens=java.base/jdk.internal.ref=ALL-UNNAMED --add-opens=java.base/sun.nio.ch=ALL-UNNAMED --add-opens=java.base/sun.nio.cs=ALL-UNNAMED --add-opens=java.base/sun.security.action=ALL-UNNAMED --add-opens=java.base/sun.util.calendar=ALL-UNNAMED --add-opens=java.security.jgss/sun.security.krb5=ALL-UNNAMED -Djdk.reflect.useDirectMethodHandle=false From e6c81399427d7f5a008cc60ee00880bbe55df110 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 11:24:58 +0000 Subject: [PATCH 28/42] changing scalatest-maven-plugin config --- sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml | 2 ++ sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml | 4 ++++ sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml | 2 ++ sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml | 2 ++ sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 2 ++ 5 files changed, 12 insertions(+) diff --git a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml index ea95cdf00179..5a2f8ff05cea 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-3_2-12/pom.xml @@ -148,6 +148,8 @@ scalatest-maven-plugin 2.1.0 + stdOut=true,verbose=true,stdErr=true + true FDEF FDEF once diff --git a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml index 25ae31a136f3..2ccc4e9002a6 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-4_2-12/pom.xml @@ -148,8 +148,12 @@ scalatest-maven-plugin 2.1.0 + stdOut=true,verbose=true,stdErr=true + true FDEF FDEF + stdOut=true,verbose=true,stdErr=true + true once true ${project.build.directory}/surefire-reports diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml index 219b0f50f262..523116a5d181 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-12/pom.xml @@ -161,6 +161,8 @@ 2.1.0 ${scalatest.argLine} + stdOut=true,verbose=true,stdErr=true + true FDEF FDEF once diff --git a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml index 10fd86e64bf4..c1177e3ec4d5 100644 --- a/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_3-5_2-13/pom.xml @@ -141,6 +141,8 @@ 2.1.0 ${scalatest.argLine} + stdOut=true,verbose=true,stdErr=true + true FDEF FDEF once diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index 31a8f614bfbe..e2a5597c2b6d 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -140,6 +140,8 @@ 2.1.0 ${scalatest.argLine} + stdOut=true,verbose=true,stdErr=true + true FDEF FDEF once From bac9710690d1bb8a7c5f9bff8eb9281a1b215c51 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 11:46:44 +0000 Subject: [PATCH 29/42] Update cosmos-emulator-matrix.json --- .../stages/cosmos-emulator-matrix.json | 32 +++++++++---------- 1 file changed, 16 insertions(+), 16 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 2823975d5ea8..8fc34b9d5789 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -19,8 +19,8 @@ "ProfileFlag": "-Pemulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "17", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 8": { "ProfileFlag": "-Plong-emulator", @@ -33,8 +33,8 @@ "ProfileFlag": "-Plong-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "17", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Encryption Emulator Only Integration Tests": { "ProfileFlag": "-Pencryption-integration", @@ -54,7 +54,7 @@ "ProfileFlag": "-Dspark-e2e_3-3", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", + "JavaTestVersion": "11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { @@ -68,7 +68,7 @@ "ProfileFlag": "-Dspark-e2e_3-4", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", + "JavaTestVersion": "11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { @@ -82,43 +82,43 @@ "ProfileFlag": "-Dspark-e2e_3-5_2-12", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "17", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { "ProfileFlag": "-Dspark-e2e_3-5_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "17", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.21", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "21", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { "ProfileFlag": "-Pkafka-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", + "JavaTestVersion": "11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { "ProfileFlag": "-Pkafka-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "JavaTestVersion": "17", + "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } From 4feef36a9e75ab69aebe493fa3d4f312a4ae3f6e Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 12:00:22 +0000 Subject: [PATCH 30/42] Update cosmos-emulator-matrix.json --- .../stages/cosmos-emulator-matrix.json | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 8fc34b9d5789..69800df5e1ba 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -19,7 +19,7 @@ "ProfileFlag": "-Pemulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "17", + "JavaTestVersion": "1.17", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 8": { @@ -33,7 +33,7 @@ "ProfileFlag": "-Plong-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "17", + "JavaTestVersion": "1.17", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Encryption Emulator Only Integration Tests": { @@ -54,7 +54,7 @@ "ProfileFlag": "-Dspark-e2e_3-3", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "11", + "JavaTestVersion": "1.11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { @@ -68,7 +68,7 @@ "ProfileFlag": "-Dspark-e2e_3-4", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "11", + "JavaTestVersion": "1.11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { @@ -82,7 +82,7 @@ "ProfileFlag": "-Dspark-e2e_3-5_2-12", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "17", + "JavaTestVersion": "1.17", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { @@ -96,28 +96,28 @@ "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "17", + "JavaTestVersion": "1.17", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "21", + "JavaTestVersion": "1.21", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { "ProfileFlag": "-Pkafka-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "11", + "JavaTestVersion": "1.11", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { "ProfileFlag": "-Pkafka-emulator", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "17", + "JavaTestVersion": "1.17", "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } From 6823c9ea936f7aaea2f8607f516090a58bc88e88 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 13:15:40 +0000 Subject: [PATCH 31/42] Trying to enable -X for mvn cmd when running cosmos integration tests --- .../templates/stages/cosmos-emulator-matrix.json | 14 +++++++------- .../templates/stages/cosmos-sdk-client.yml | 6 ++++++ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 69800df5e1ba..2823975d5ea8 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -20,7 +20,7 @@ "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 8": { "ProfileFlag": "-Plong-emulator", @@ -34,7 +34,7 @@ "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Strong\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Encryption Emulator Only Integration Tests": { "ProfileFlag": "-Pencryption-integration", @@ -83,28 +83,28 @@ "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { "ProfileFlag": "-Dspark-e2e_3-5_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.21", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { "ProfileFlag": "-Pkafka-emulator", @@ -118,7 +118,7 @@ "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.17", - "AdditionalArgs": "-X -DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index 6ff37f7f4a6d..b022acec930c 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -29,6 +29,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml + - name: DefaultTestOptions + value: '$(DefaultTestOptions) -X' # Convert artifact parameter objects to json and set them as variables to be used in # pipeline jobs/stages. By setting these as a variable once we save thousands of lines # of yaml in extreme cases. This helps us stay under the devops max yaml size limits. @@ -70,6 +72,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml + - name: DefaultTestOptions + value: '$(DefaultTestOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson @@ -134,6 +138,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml + - name: DefaultTestOptions + value: '$(DefaultTestOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson From 8cd221395ff45677242729cfec82d8ae0ad5e4c8 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 13:27:11 +0000 Subject: [PATCH 32/42] Update cosmos-sdk-client.yml --- eng/pipelines/templates/stages/cosmos-sdk-client.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index b022acec930c..a3a1a60b6317 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -29,8 +29,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultTestOptions - value: '$(DefaultTestOptions) -X' + - name: DefaultOptions + value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' # Convert artifact parameter objects to json and set them as variables to be used in # pipeline jobs/stages. By setting these as a variable once we save thousands of lines # of yaml in extreme cases. This helps us stay under the devops max yaml size limits. @@ -72,8 +72,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultTestOptions - value: '$(DefaultTestOptions) -X' + - name: DefaultOptions + value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson @@ -138,8 +138,8 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultTestOptions - value: '$(DefaultTestOptions) -X' + - name: DefaultOptions + value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson From ac9418018320a1d94ff9f6e4ca3cf7b5d27ea390 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 15:59:48 +0000 Subject: [PATCH 33/42] Update cosmos-sdk-client.yml --- .../templates/stages/cosmos-sdk-client.yml | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index a3a1a60b6317..46f6ffc27811 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -102,6 +102,34 @@ extends: - template: /eng/common/pipelines/templates/steps/cosmos-emulator.yml parameters: StartParameters: '/noexplorer /noui /enablepreview /EnableSqlComputeEndpoint /SqlComputePort=9999 /disableratelimiting /partitioncount=50 /consistency=Strong' + PostSteps: + - pwsh: | + Write-Host "===== SCALATEST REPORTS (TXT) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.txt" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST REPORTS (OUT) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.out" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST REPORTS (ERR) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.err" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST XML (RAW) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.xml" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + displayName: "Print ScalaTest report files" + errorActionPreference: continue + condition: always() - powershell: | $Key = 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw==' $password = ConvertTo-SecureString -String $Key -Force -AsPlainText @@ -189,3 +217,5 @@ extends: - ${{ if ne(artifact.releaseInBatch, 'false') }}: - ${{ artifact }} TestPipeline: false + + From 77aac608445fe61a6667b65991d85ec1a9ade885 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 17:17:36 +0000 Subject: [PATCH 34/42] Debugging tests --- .../stages/cosmos-emulator-matrix.json | 105 ------------------ .../templates/stages/cosmos-sdk-client.yml | 57 +++++----- 2 files changed, 28 insertions(+), 134 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 2823975d5ea8..79e163fab7c1 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -8,117 +8,12 @@ }, "TestGoals": "clean verify", "EmulatorConfig": { - "Emulator Only Integration Tests Tcp - Java 8": { - "ProfileFlag": "-Pemulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Pemulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Long Emulator Only Integration Tests Tcp - Java 8": { - "ProfileFlag": "-Plong-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Long Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Plong-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Encryption Emulator Only Integration Tests": { - "ProfileFlag": "-Pencryption-integration", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_4-0_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.21", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" - }, - "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index 46f6ffc27811..43ce9dae85fb 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -102,34 +102,6 @@ extends: - template: /eng/common/pipelines/templates/steps/cosmos-emulator.yml parameters: StartParameters: '/noexplorer /noui /enablepreview /EnableSqlComputeEndpoint /SqlComputePort=9999 /disableratelimiting /partitioncount=50 /consistency=Strong' - PostSteps: - - pwsh: | - Write-Host "===== SCALATEST REPORTS (TXT) =====" - Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.txt" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { - Write-Host "File: $($_.FullName)" - Get-Content $_.FullName - } - - Write-Host "===== SCALATEST REPORTS (OUT) =====" - Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.out" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { - Write-Host "File: $($_.FullName)" - Get-Content $_.FullName - } - - Write-Host "===== SCALATEST REPORTS (ERR) =====" - Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.err" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { - Write-Host "File: $($_.FullName)" - Get-Content $_.FullName - } - - Write-Host "===== SCALATEST XML (RAW) =====" - Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.xml" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { - Write-Host "File: $($_.FullName)" - Get-Content $_.FullName - } - displayName: "Print ScalaTest report files" - errorActionPreference: continue - condition: always() - powershell: | $Key = 'C2y6yDjf5/R+ob0N8A7Cgv30VRDJIWEHLM+4QDU5DE2nQ9nDuVTqobD4b8mGGyPMbIZnqyMsEcaGQy67XIw/Jw==' $password = ConvertTo-SecureString -String $Key -Force -AsPlainText @@ -157,7 +129,34 @@ extends: keytool -keystore cacerts -delete -noprompt -alias CosmosDbEmulatorCert -storepass changeit keytool -keystore cacerts -importcert -noprompt -trustcacerts -alias CosmosDbEmulatorCert -file CosmosDbEmulatorCert.cer -storepass changeit displayName: 'Create Java TrustStore' - + PostSteps: + - powershell: | + Write-Host "===== SCALATEST REPORTS (TXT) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.txt" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST REPORTS (OUT) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.out" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST REPORTS (ERR) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.err" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + + Write-Host "===== SCALATEST XML (RAW) =====" + Get-ChildItem -Path "$(System.DefaultWorkingDirectory)" -Recurse -Filter "*.xml" | Where-Object { $_.FullName -like "*surefire-reports*" } | ForEach-Object { + Write-Host "File: $($_.FullName)" + Get-Content $_.FullName + } + displayName: "Print ScalaTest report files" + errorActionPreference: continue + condition: always() - ${{ if eq(variables['System.TeamProject'], 'internal') }}: - ${{ each mode in parameters.VnextEmulatorModes }}: - stage: From 32eb38dd81d3a0e47cd6490b17fc442e1655ade8 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 19:00:55 +0000 Subject: [PATCH 35/42] Adding missing hive test dependency --- eng/versioning/external_dependencies.txt | 1 + sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 3df1ee050073..3adfb3e6a6f7 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -252,6 +252,7 @@ cosmos-spark_3-3_org.apache.spark:spark-hive_2.12;3.3.0 cosmos-spark_3-4_org.apache.spark:spark-hive_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-hive_2.12;3.5.0 cosmos-spark_4-0_org.apache.spark:spark-hive_2.13;4.0.0 +cosmos-spark_4-0_org.apache.derby:derby;10.17.1.0 cosmos_org.scala-lang:scala-library;2.12.19 cosmos-scala213_org.scala-lang:scala-library;2.13.17 cosmos_org.scala-lang.modules:scala-java8-compat_2.12;0.9.1 diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index e2a5597c2b6d..25f3600b48d4 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -226,5 +226,11 @@ jackson-module-scala_2.13 2.18.4 + + org.apache.derby + derby + 10.17.1.0 + test + From b3b1e65da70e1fc95c8131ea8b48655ac41b567a Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Tue, 20 Jan 2026 22:39:51 +0000 Subject: [PATCH 36/42] Adding derbytools dependency --- eng/versioning/external_dependencies.txt | 1 + sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml | 6 ++++++ 2 files changed, 7 insertions(+) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 3adfb3e6a6f7..1045a78dc7aa 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -253,6 +253,7 @@ cosmos-spark_3-4_org.apache.spark:spark-hive_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-hive_2.12;3.5.0 cosmos-spark_4-0_org.apache.spark:spark-hive_2.13;4.0.0 cosmos-spark_4-0_org.apache.derby:derby;10.17.1.0 +cosmos-spark_4-0_org.apache.derby:derbytools;10.17.1.0 cosmos_org.scala-lang:scala-library;2.12.19 cosmos-scala213_org.scala-lang:scala-library;2.13.17 cosmos_org.scala-lang.modules:scala-java8-compat_2.12;0.9.1 diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index 25f3600b48d4..02b93a83582e 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -232,5 +232,11 @@ 10.17.1.0 test + + org.apache.derby + derbytools + 10.17.1.0 + test + From dc323b970a9accc3f9c24d2dd2372ee640e3fa2a Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:17:03 +0000 Subject: [PATCH 37/42] Fixing unit tests --- .../stages/cosmos-emulator-matrix.json | 105 ++++++++++++++++++ .../templates/stages/cosmos-sdk-client.yml | 6 - eng/versioning/external_dependencies.txt | 2 - .../cosmos/spark/CosmosCatalogITestBase.scala | 12 +- .../azure-cosmos-spark_4-0_2-13/pom.xml | 28 ----- .../cosmos/spark/CosmosCatalogITest.scala | 7 +- 6 files changed, 117 insertions(+), 43 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 79e163fab7c1..9c772340110f 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -8,12 +8,117 @@ }, "TestGoals": "clean verify", "EmulatorConfig": { + "Emulator Only Integration Tests Tcp - Java 8": { + "ProfileFlag": "-Pemulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Emulator Only Integration Tests Tcp - Java 17": { + "ProfileFlag": "-Pemulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Long Emulator Only Integration Tests Tcp - Java 8": { + "ProfileFlag": "-Plong-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Long Emulator Only Integration Tests Tcp - Java 17": { + "ProfileFlag": "-Plong-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Encryption Emulator Only Integration Tests": { + "ProfileFlag": "-Pencryption-integration", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 8'": { + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 11'": { + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 11'": { + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 17'": { + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { + "ProfileFlag": "-Dspark-e2e_3-5_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { "ProfileFlag": "-Dspark-e2e_4-0_2-13", "PROTOCOLS": "[\"Tcp\"]", "DESIRED_CONSISTENCIES": "[\"Session\"]", "JavaTestVersion": "1.21", "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + }, + "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index 43ce9dae85fb..6d6ca8885bad 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -29,8 +29,6 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultOptions - value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' # Convert artifact parameter objects to json and set them as variables to be used in # pipeline jobs/stages. By setting these as a variable once we save thousands of lines # of yaml in extreme cases. This helps us stay under the devops max yaml size limits. @@ -72,8 +70,6 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultOptions - value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson @@ -165,8 +161,6 @@ extends: variables: - template: /eng/pipelines/templates/variables/globals.yml - template: /eng/pipelines/templates/variables/image.yml - - name: DefaultOptions - value: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER) --batch-mode --fail-at-end --settings eng/settings.xml $(WagonOptions) -X' - name: ArtifactsJson value: '${{ convertToJson(parameters.Artifacts) }}' - name: AdditionalModulesJson diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 1045a78dc7aa..3df1ee050073 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -252,8 +252,6 @@ cosmos-spark_3-3_org.apache.spark:spark-hive_2.12;3.3.0 cosmos-spark_3-4_org.apache.spark:spark-hive_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-hive_2.12;3.5.0 cosmos-spark_4-0_org.apache.spark:spark-hive_2.13;4.0.0 -cosmos-spark_4-0_org.apache.derby:derby;10.17.1.0 -cosmos-spark_4-0_org.apache.derby:derbytools;10.17.1.0 cosmos_org.scala-lang:scala-library;2.12.19 cosmos-scala213_org.scala-lang:scala-library;2.13.17 cosmos_org.scala-lang.modules:scala-java8-compat_2.12;0.9.1 diff --git a/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITestBase.scala b/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITestBase.scala index eaeb7bec55b9..78272dda2227 100644 --- a/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITestBase.scala +++ b/sdk/cosmos/azure-cosmos-spark_3/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITestBase.scala @@ -14,7 +14,7 @@ import java.util.UUID import scala.collection.JavaConverters._ // scalastyle:on underscore.import -abstract class CosmosCatalogITestBase extends IntegrationSpec with CosmosClient with BasicLoggingTrait { +abstract class CosmosCatalogITestBase(val skipHive: Boolean = false) extends IntegrationSpec with CosmosClient with BasicLoggingTrait { //scalastyle:off multiple.string.literals //scalastyle:off magic.number @@ -25,11 +25,15 @@ abstract class CosmosCatalogITestBase extends IntegrationSpec with CosmosClient val cosmosEndpoint = TestConfigurations.HOST val cosmosMasterKey = TestConfigurations.MASTER_KEY - spark = SparkSession.builder() + var sparkBuilder = SparkSession.builder() .appName("spark connector sample") .master("local") - .enableHiveSupport() - .getOrCreate() + + if (!skipHive) { + sparkBuilder = sparkBuilder.enableHiveSupport() + } + + spark = sparkBuilder.getOrCreate() LocalJavaFileSystem.applyToSparkSession(spark) diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml index 02b93a83582e..f8ea117123a9 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/pom.xml @@ -200,22 +200,6 @@ provided - - org.apache.spark - spark-hive_2.13 - 4.0.0 - - - io.netty - netty-all - - - org.slf4j - * - - - test - com.fasterxml.jackson.core jackson-databind @@ -226,17 +210,5 @@ jackson-module-scala_2.13 2.18.4 - - org.apache.derby - derby - 10.17.1.0 - test - - - org.apache.derby - derbytools - 10.17.1.0 - test - diff --git a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala index c423992f61cb..a7219b159068 100644 --- a/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala +++ b/sdk/cosmos/azure-cosmos-spark_4-0_2-13/src/test/scala/com/azure/cosmos/spark/CosmosCatalogITest.scala @@ -8,7 +8,7 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.NonEmptyNamespaceException class CosmosCatalogITest - extends CosmosCatalogITestBase { + extends CosmosCatalogITestBase(skipHive = true) { //scalastyle:off magic.number @@ -31,8 +31,6 @@ class CosmosCatalogITest } it can "drop an non-empty database with cascade true" in { - assume(!Platform.isWindows) - val databaseName = getAutoCleanableDatabaseName spark.catalog.databaseExists(databaseName) shouldEqual false @@ -46,6 +44,9 @@ class CosmosCatalogITest spark.catalog.databaseExists(databaseName) shouldEqual false } + // TODO: spark on windows has issue with this test. + // java.lang.RuntimeException: java.io.IOException: (null) entry in command string: null chmod 0733 D:\tmp\hive; + // once we move Linux CI re-enable the test: "drop an non-empty database with cascade false" should "throw NonEmptyNamespaceException" in { assume(!Platform.isWindows) From 3dca44ceed777ca8627aacb0d6a9fca68cc3bbf0 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:20:07 +0000 Subject: [PATCH 38/42] Update cosmos-emulator-matrix.json --- .../stages/cosmos-emulator-matrix.json | 160 +++++++++--------- 1 file changed, 80 insertions(+), 80 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index 9c772340110f..fbcca6325131 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -9,116 +9,116 @@ "TestGoals": "clean verify", "EmulatorConfig": { "Emulator Only Integration Tests Tcp - Java 8": { - "ProfileFlag": "-Pemulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pemulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Pemulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pemulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 8": { - "ProfileFlag": "-Plong-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Plong-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Plong-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Plong-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Encryption Emulator Only Integration Tests": { - "ProfileFlag": "-Pencryption-integration", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pencryption-integration", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_4-0_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { - "ProfileFlag": "-Dspark-e2e_4-0_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.21", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.21", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } From ad55f00af9a9e68be42fdaa330a0a1908d44e9fc Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:23:55 +0000 Subject: [PATCH 39/42] Update cosmos-emulator-matrix.json --- .../stages/cosmos-emulator-matrix.json | 130 +++++++++--------- 1 file changed, 65 insertions(+), 65 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json index fbcca6325131..2823975d5ea8 100644 --- a/eng/pipelines/templates/stages/cosmos-emulator-matrix.json +++ b/eng/pipelines/templates/stages/cosmos-emulator-matrix.json @@ -16,11 +16,11 @@ "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Pemulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pemulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 8": { "ProfileFlag": "-Plong-emulator", @@ -30,11 +30,11 @@ "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Long Emulator Only Integration Tests Tcp - Java 17": { - "ProfileFlag": "-Plong-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Strong\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Plong-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Strong\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Encryption Emulator Only Integration Tests": { "ProfileFlag": "-Pencryption-integration", @@ -44,81 +44,81 @@ "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.3 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-3", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-3", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.4 Integration Tests targeting Cosmos Emulator - Java 11'": { - "ProfileFlag": "-Dspark-e2e_3-4", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-4", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 8'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.8", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.8", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.12 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-12", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-12", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 3.5, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_3-5_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_3-5_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 17'": { - "ProfileFlag": "-Dspark-e2e_4-0_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Spark 4.0, Scala 2.13 Integration Tests targeting Cosmos Emulator - Java 21'": { - "ProfileFlag": "-Dspark-e2e_4-0_2-13", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.21", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Dspark-e2e_4-0_2-13", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.21", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -Dhadoop.home.dir=D:/Hadoop -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 11": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.11", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.11", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" }, "Kafka Integration Tests targeting Cosmos Emulator - Java 17": { - "ProfileFlag": "-Pkafka-emulator", - "PROTOCOLS": "[\"Tcp\"]", - "DESIRED_CONSISTENCIES": "[\"Session\"]", - "JavaTestVersion": "1.17", - "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" + "ProfileFlag": "-Pkafka-emulator", + "PROTOCOLS": "[\"Tcp\"]", + "DESIRED_CONSISTENCIES": "[\"Session\"]", + "JavaTestVersion": "1.17", + "AdditionalArgs": "-DACCOUNT_HOST=https://localhost:8081/ -DCOSMOS.AZURE_COSMOS_DISABLE_NON_STREAMING_ORDER_BY=true" } } } From ecce3909e897f12bef82ab0044fe393cb1264ab9 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:25:07 +0000 Subject: [PATCH 40/42] Update cosmos-sdk-client.yml --- eng/pipelines/templates/stages/cosmos-sdk-client.yml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index 6d6ca8885bad..9ea98dd89785 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -209,6 +209,4 @@ extends: - ${{ each artifact in parameters.Artifacts }}: - ${{ if ne(artifact.releaseInBatch, 'false') }}: - ${{ artifact }} - TestPipeline: false - - + TestPipeline: false \ No newline at end of file From 796c30bb730231454c0916688df60707ef3d7aa2 Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:25:43 +0000 Subject: [PATCH 41/42] Update cosmos-sdk-client.yml --- eng/pipelines/templates/stages/cosmos-sdk-client.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/eng/pipelines/templates/stages/cosmos-sdk-client.yml b/eng/pipelines/templates/stages/cosmos-sdk-client.yml index 9ea98dd89785..72ead8db96a8 100644 --- a/eng/pipelines/templates/stages/cosmos-sdk-client.yml +++ b/eng/pipelines/templates/stages/cosmos-sdk-client.yml @@ -209,4 +209,4 @@ extends: - ${{ each artifact in parameters.Artifacts }}: - ${{ if ne(artifact.releaseInBatch, 'false') }}: - ${{ artifact }} - TestPipeline: false \ No newline at end of file + TestPipeline: false From afaa10222ec0686dda281023ee0c036b1176545d Mon Sep 17 00:00:00 2001 From: Fabian Meiswinkel Date: Wed, 21 Jan 2026 00:27:54 +0000 Subject: [PATCH 42/42] Update external_dependencies.txt --- eng/versioning/external_dependencies.txt | 1 - 1 file changed, 1 deletion(-) diff --git a/eng/versioning/external_dependencies.txt b/eng/versioning/external_dependencies.txt index 3df1ee050073..07cd02ef9c7c 100644 --- a/eng/versioning/external_dependencies.txt +++ b/eng/versioning/external_dependencies.txt @@ -251,7 +251,6 @@ cosmos-spark_4-0_org.apache.spark:spark-sql_2.13;4.0.0 cosmos-spark_3-3_org.apache.spark:spark-hive_2.12;3.3.0 cosmos-spark_3-4_org.apache.spark:spark-hive_2.12;3.4.0 cosmos-spark_3-5_org.apache.spark:spark-hive_2.12;3.5.0 -cosmos-spark_4-0_org.apache.spark:spark-hive_2.13;4.0.0 cosmos_org.scala-lang:scala-library;2.12.19 cosmos-scala213_org.scala-lang:scala-library;2.13.17 cosmos_org.scala-lang.modules:scala-java8-compat_2.12;0.9.1