Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

azure-cosmos-spark: release 2024-03-12 - Reducing noisy logs in WARN but adding more verbose logs when retries have been re-enqueued. #39169

Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,13 @@ package com.azure.cosmos.implementation

import com.azure.cosmos.implementation.ImplementationBridgeHelpers.CosmosClientBuilderHelper
import com.azure.cosmos.implementation.changefeed.common.{ChangeFeedMode, ChangeFeedStartFromInternal, ChangeFeedState, ChangeFeedStateV1}
import com.azure.cosmos.implementation.guava25.base.MoreObjects.firstNonNull
import com.azure.cosmos.implementation.guava25.base.Strings.emptyToNull
import com.azure.cosmos.implementation.query.CompositeContinuationToken
import com.azure.cosmos.implementation.routing.Range
import com.azure.cosmos.models.{FeedRange, PartitionKey, PartitionKeyBuilder, PartitionKeyDefinition, SparkModelBridgeInternal}
import com.azure.cosmos.spark.diagnostics.BasicLoggingTrait
import com.azure.cosmos.spark.{ChangeFeedOffset, NormalizedRange}
import com.azure.cosmos.spark.{ChangeFeedOffset, CosmosConstants, NormalizedRange}
import com.azure.cosmos.{CosmosAsyncClient, CosmosClientBuilder, DirectConnectionConfig, SparkBridgeInternal}
import com.fasterxml.jackson.databind.ObjectMapper

Expand All @@ -21,6 +23,14 @@ import scala.collection.JavaConverters._
// scalastyle:on underscore.import

private[cosmos] object SparkBridgeImplementationInternal extends BasicLoggingTrait {
private val SPARK_MAX_CONNECTIONS_PER_ENDPOINT_PROPERTY = "COSMOS.SPARK_MAX_CONNECTIONS_PER_ENDPOINT"
private val SPARK_MAX_CONNECTIONS_PER_ENDPOINT_VARIABLE = "COSMOS_SPARK_MAX_CONNECTIONS_PER_ENDPOINT"
private val DEFAULT_SPARK_MAX_CONNECTIONS_PER_ENDPOINT: Int = DirectConnectionConfig.getDefaultConfig.getMaxConnectionsPerEndpoint

private val SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE_PROPERTY = "COSMOS.SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE"
private val SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE_VARIABLE = "COSMOS_SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE"
private val DEFAULT_SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE: Int = CosmosConstants.defaultIoThreadCountFactorPerCore

def setMetadataCacheSnapshot(cosmosClientBuilder: CosmosClientBuilder,
metadataCache: CosmosClientMetadataCachesSnapshot): Unit = {

Expand Down Expand Up @@ -356,6 +366,41 @@ private[cosmos] object SparkBridgeImplementationInternal extends BasicLoggingTra
).json()
}

private def getMaxConnectionsPerEndpointOverride: Int = {
val maxConnectionsPerEndpointText = System.getProperty(
SPARK_MAX_CONNECTIONS_PER_ENDPOINT_PROPERTY,
firstNonNull(
emptyToNull(System.getenv.get(SPARK_MAX_CONNECTIONS_PER_ENDPOINT_VARIABLE)),
String.valueOf(DEFAULT_SPARK_MAX_CONNECTIONS_PER_ENDPOINT)))

try {
maxConnectionsPerEndpointText.toInt
}
catch {
case e: Exception =>
logError(s"Parsing spark max connections per endpoint failed. Using the default $DEFAULT_SPARK_MAX_CONNECTIONS_PER_ENDPOINT.", e)
DEFAULT_SPARK_MAX_CONNECTIONS_PER_ENDPOINT
}
}

def getIoThreadCountPerCoreOverride: Int = {
val ioThreadCountPerCoreText = System.getProperty(
SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE_PROPERTY,
firstNonNull(
emptyToNull(System.getenv.get(SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE_VARIABLE)),
String.valueOf(DEFAULT_SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE)))

try {
ioThreadCountPerCoreText.toInt
}
catch {
case e: Exception =>
logError(s"Parsing spark I/O thread-count per core failed. Using the default $DEFAULT_SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE.", e)
DEFAULT_SPARK_IO_THREAD_COUNT_FACTOR_PER_CORE
}
}


def configureSimpleObjectMapper(allowDuplicateProperties: Boolean) : Unit = {
Utils.configureSimpleObjectMapper(allowDuplicateProperties)
}
Expand All @@ -365,22 +410,24 @@ private[cosmos] object SparkBridgeImplementationInternal extends BasicLoggingTra
"\"timeoutDetectionTimeLimit\": \"PT90S\", \"timeoutDetectionHighFrequencyThreshold\": 10," +
"\"timeoutDetectionHighFrequencyTimeLimit\": \"PT30S\", \"timeoutDetectionOnWriteThreshold\": 10," +
"\"timeoutDetectionOnWriteTimeLimit\": \"PT90s\", \"tcpNetworkRequestTimeout\": \"PT7S\", " +
"\"connectTimeout\": \"PT10S\"}"
"\"connectTimeout\": \"PT10S\", \"maxChannelsPerEndpoint\": \"" +
s"$getMaxConnectionsPerEndpointOverride" +
"\"}"

if (System.getProperty("reactor.netty.tcp.sslHandshakeTimeout") == null) {
System.setProperty("reactor.netty.tcp.sslHandshakeTimeout", "20000");
System.setProperty("reactor.netty.tcp.sslHandshakeTimeout", "20000")
}

if (System.getProperty(Configs.HTTP_MAX_REQUEST_TIMEOUT) == null) {
System.setProperty(
Configs.HTTP_MAX_REQUEST_TIMEOUT,
"70");
"70")
}

if (System.getProperty(Configs.HTTP_DEFAULT_CONNECTION_POOL_SIZE) == null) {
System.setProperty(
Configs.HTTP_DEFAULT_CONNECTION_POOL_SIZE,
"25000");
"25000")
}

if (System.getProperty("azure.cosmos.directTcp.defaultOptions") == null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import com.azure.cosmos.implementation.apachecommons.lang.StringUtils
import com.azure.cosmos.implementation.batch.{BatchRequestResponseConstants, BulkExecutorDiagnosticsTracker, ItemBulkOperation}
import com.azure.cosmos.models._
import com.azure.cosmos.spark.BulkWriter.{BulkOperationFailedException, bulkWriterBoundedElastic, getThreadInfo, readManyBoundedElastic}
import com.azure.cosmos.spark.CosmosConstants.StatusCodes
import com.azure.cosmos.spark.diagnostics.DefaultDiagnostics
import reactor.core.publisher.Mono
import reactor.core.scheduler.Scheduler
Expand Down Expand Up @@ -55,6 +56,8 @@ private class BulkWriter(container: CosmosAsyncContainer,

private val log = LoggerHelper.getLogger(diagnosticsConfig, this.getClass)

private val verboseLoggingAfterReEnqueueingRetriesEnabled = new AtomicBoolean(false)

private val cpuCount = SparkUtils.getNumberOfHostCPUCores
// each bulk writer allows up to maxPendingOperations being buffered
// there is one bulk writer per spark task/partition
Expand Down Expand Up @@ -111,17 +114,25 @@ private class BulkWriter(container: CosmosAsyncContainer,
maxConcurrentPartitions
)

private class ForwardingMetricTracker extends BulkExecutorDiagnosticsTracker {
private class ForwardingMetricTracker(val verboseLoggingEnabled: AtomicBoolean) extends BulkExecutorDiagnosticsTracker {
override def trackDiagnostics(ctx: CosmosDiagnosticsContext): Unit = {
outputMetricsPublisher.trackWriteOperation(0, Option.apply(ctx))
val ctxOption = Option.apply(ctx)
outputMetricsPublisher.trackWriteOperation(0, ctxOption)
if (ctxOption.isDefined && verboseLoggingEnabled.get) {
BulkWriter.log.logWarning(s"Track bulk operation after re-enqueued retry: ${ctxOption.get.toJson}")
}
}

override def verboseLoggingAfterReEnqueueingRetriesEnabled(): Boolean = {
verboseLoggingEnabled.get()
}
}

ImplementationBridgeHelpers.CosmosBulkExecutionOptionsHelper
.getCosmosBulkExecutionOptionsAccessor
.setDiagnosticsTracker(
cosmosBulkExecutionOptions,
new ForwardingMetricTracker
new ForwardingMetricTracker(verboseLoggingAfterReEnqueueingRetriesEnabled)
)

ThroughputControlHelper.populateThroughputControlGroupName(cosmosBulkExecutionOptions, writeConfig.throughputControlConfig)
Expand Down Expand Up @@ -421,7 +432,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
// so we are not going to make task complete here
if (!activeReadManyOperationFound) {
// can't find the read-many operation in list of active operations!
log.logInfo(s"Cannot find active read-many for '"
logInfoOrWarning(s"Cannot find active read-many for '"
+ s"${readManyOperation.cosmosItemIdentity.getPartitionKey}/"
+ s"${readManyOperation.cosmosItemIdentity.getId}'. This can happen when "
+ s"retries get re-enqueued.")
Expand Down Expand Up @@ -563,7 +574,7 @@ private class BulkWriter(container: CosmosAsyncContainer,

if (!itemOperationFound) {
// can't find the item operation in list of active operations!
log.logInfo(s"Cannot find active operation for '${itemOperation.getOperationType} " +
logInfoOrWarning(s"Cannot find active operation for '${itemOperation.getOperationType} " +
s"${itemOperation.getPartitionKeyValue}/${itemOperation.getId}'. This can happen when " +
s"retries get re-enqueued.")
shouldSkipTaskCompletion.set(true)
Expand Down Expand Up @@ -668,7 +679,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
operationContext: OperationContext): Unit = {
activeTasks.incrementAndGet()
if (operationContext.attemptNumber > 1) {
log.logInfo(s"bulk scheduleWrite attemptCnt: ${operationContext.attemptNumber}, " +
logInfoOrWarning(s"bulk scheduleWrite attemptCnt: ${operationContext.attemptNumber}, " +
s"Context: ${operationContext.toString} $getThreadInfo")
}

Expand Down Expand Up @@ -923,7 +934,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
)
} else {
numberOfIntervalsWithIdenticalActiveOperationSnapshots.set(0)
log.logInfo(
logInfoOrWarning(
s"$operationName is waiting for active bulkWrite operations: $operationsLog " +
s"Context: ${operationContext.toString} $getThreadInfo"
)
Expand Down Expand Up @@ -966,7 +977,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
while ((pendingRetriesSnapshot > 0 || activeTasksSnapshot > 0)
&& errorCaptureFirstException.get == null) {

log.logInfo(
logInfoOrWarning(
s"Waiting for pending activeTasks $activeTasksSnapshot and/or pendingRetries " +
s"$pendingRetriesSnapshot, Context: ${operationContext.toString} $getThreadInfo")
val activeOperationsSnapshot = activeBulkWriteOperations.clone()
Expand All @@ -980,19 +991,27 @@ private class BulkWriter(container: CosmosAsyncContainer,
numberOfIntervalsWithIdenticalActiveOperationSnapshots
)

if (numberOfIntervalsWithIdenticalActiveOperationSnapshots.get == 1L &&
if (numberOfIntervalsWithIdenticalActiveOperationSnapshots.get <= 10L &&
FabianMeiswinkel marked this conversation as resolved.
Show resolved Hide resolved
// pending retries only tracks the time to enqueue
// the retry - so this should never take longer than 1 minute
pendingRetriesSnapshot == 0L) {

if (verboseLoggingAfterReEnqueueingRetriesEnabled.compareAndSet(false, true)) {
FabianMeiswinkel marked this conversation as resolved.
Show resolved Hide resolved
log.logWarning(s"Starting to re-enqueue retries. Enabling verbose logs. "
+ s"Attempt: ${numberOfIntervalsWithIdenticalActiveOperationSnapshots.get} - "
+ s"Context: ${operationContext.toString} $getThreadInfo")
}

activeOperationsSnapshot.foreach(operation => {
if (activeBulkWriteOperations.contains(operation)) {
// re-validating whether the operation is still active - if so, just re-enqueue another retry
// this is harmless - because all bulkItemOperations from Spark connector are always idempotent

// For FAIL_NON_SERIALIZED, will keep retry, while for other errors, use the default behavior
bulkInputEmitter.emitNext(operation, emitFailureHandler)
log.logInfo(s"Re-enqueued a retry for pending active write task '${operation.getOperationType} "
log.logWarning(s"Re-enqueued a retry for pending active write task '${operation.getOperationType} "
+ s"(${operation.getPartitionKeyValue}/${operation.getId})' "
+s"- Attempt: ${numberOfIntervalsWithIdenticalActiveOperationSnapshots.get} - "
+ s"Context: ${operationContext.toString} $getThreadInfo")
}
})
Expand All @@ -1004,8 +1023,9 @@ private class BulkWriter(container: CosmosAsyncContainer,

// For FAIL_NON_SERIALIZED, will keep retry, while for other errors, use the default behavior
readManyInputEmitterOpt.get.emitNext(operation, emitFailureHandler)
log.logInfo(s"Re-enqueued a retry for pending active read-many task '"
log.logWarning(s"Re-enqueued a retry for pending active read-many task '"
+ s"(${operation.cosmosItemIdentity.getPartitionKey}/${operation.cosmosItemIdentity.getId})' "
+ s"- Attempt: ${numberOfIntervalsWithIdenticalActiveOperationSnapshots.get} - "
+ s"Context: ${operationContext.toString} $getThreadInfo")
}
})
Expand All @@ -1017,44 +1037,28 @@ private class BulkWriter(container: CosmosAsyncContainer,
val semaphoreAvailablePermitsSnapshot = semaphore.availablePermits()

if (awaitCompleted) {
log.logInfo(s"Waiting completed for pending activeTasks $activeTasksSnapshot, pendingRetries " +
logInfoOrWarning(s"Waiting completed for pending activeTasks $activeTasksSnapshot, pendingRetries " +
s"$pendingRetriesSnapshot Context: ${operationContext.toString} $getThreadInfo")
} else {
log.logInfo(s"Waiting interrupted for pending activeTasks $activeTasksSnapshot , pendingRetries " +
logInfoOrWarning(s"Waiting interrupted for pending activeTasks $activeTasksSnapshot , pendingRetries " +
s"$pendingRetriesSnapshot - available permits $semaphoreAvailablePermitsSnapshot, " +
s"Context: ${operationContext.toString} $getThreadInfo")
}
}

log.logInfo(s"Waiting completed for pending activeTasks $activeTasksSnapshot, pendingRetries " +
logInfoOrWarning(s"Waiting completed for pending activeTasks $activeTasksSnapshot, pendingRetries " +
s"$pendingRetriesSnapshot Context: ${operationContext.toString} $getThreadInfo")
} finally {
lock.unlock()
}

log.logInfo(s"invoking bulkInputEmitter.onComplete(), Context: ${operationContext.toString} $getThreadInfo")
logInfoOrWarning(s"invoking bulkInputEmitter.onComplete(), Context: ${operationContext.toString} $getThreadInfo")
semaphore.release(Math.max(0, activeTasks.get()))
val completeBulkWriteEmitResult = bulkInputEmitter.tryEmitComplete()
if (completeBulkWriteEmitResult eq Sinks.EmitResult.OK) {
log.logDebug(s"bulkInputEmitter sink completed, Context: ${operationContext.toString} $getThreadInfo")
}
else {
log.logInfo(
s"bulkInputEmitter sink completion failed. EmitResult: $completeBulkWriteEmitResult +" +
s"Context: ${operationContext.toString} $getThreadInfo")
}
val completeBulkWriteEmitResult = bulkInputEmitter.emitComplete(BulkWriter.emitFailureHandlerForComplete)

// complete readManyInputEmitter
if (readManyInputEmitterOpt.isDefined) {
val completeReadManyEmitResult = readManyInputEmitterOpt.get.tryEmitComplete()
if (completeReadManyEmitResult eq Sinks.EmitResult.OK) {
log.logDebug(s"bulkInputEmitter sink completed, Context: ${operationContext.toString} $getThreadInfo")
}
else {
log.logInfo(
s"bulkInputEmitter sink completion failed. EmitResult: $completeReadManyEmitResult +" +
s"Context: ${operationContext.toString} $getThreadInfo")
}
val completeReadManyEmitResult = readManyInputEmitterOpt.get.emitComplete(BulkWriter.emitFailureHandlerForComplete)
}

throwIfCapturedExceptionExists()
Expand All @@ -1070,7 +1074,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
s"retries. totalSuccessfulIngestionMetrics=${totalSuccessfulIngestionMetrics.get()}, " +
s"totalScheduled=$totalScheduledMetrics, Context: ${operationContext.toString} $getThreadInfo")
} else {
log.logInfo(s"flushAndClose completed with no error. " +
logInfoOrWarning(s"flushAndClose completed with no error. " +
s"totalSuccessfulIngestionMetrics=${totalSuccessfulIngestionMetrics.get()}, " +
s"totalScheduled=$totalScheduledMetrics, Context: ${operationContext.toString} $getThreadInfo")
}
Expand Down Expand Up @@ -1115,6 +1119,14 @@ private class BulkWriter(container: CosmosAsyncContainer,
// scalastyle:on method.length
// scalastyle:on cyclomatic.complexity

private def logInfoOrWarning(msg: => String): Unit = {
if (this.verboseLoggingAfterReEnqueueingRetriesEnabled.get()) {
log.logWarning(msg)
} else {
log.logInfo(msg)
}
}

private def markTaskCompletion(): Unit = {
lock.lock()
try {
Expand Down Expand Up @@ -1142,7 +1154,7 @@ private class BulkWriter(container: CosmosAsyncContainer,
}

private def cancelWork(): Unit = {
log.logInfo(s"cancelling remaining unprocessed tasks ${activeTasks.get} " +
logInfoOrWarning(s"cancelling remaining unprocessed tasks ${activeTasks.get} " +
s"[bulkWrite tasks ${activeBulkWriteOperations.size}, readMany tasks ${activeReadManyOperations.size} ]" +
s"Context: ${operationContext.toString}")
subscriptionDisposable.dispose()
Expand Down Expand Up @@ -1173,9 +1185,14 @@ private class BulkWriter(container: CosmosAsyncContainer,
returnValue = writeConfig.itemWriteStrategy match {
case ItemWriteStrategy.ItemBulkUpdate =>
this.shouldRetryForItemPatchBulkUpdate(statusCode, subStatusCode)
case _ => Exceptions.canBeTransientFailure(statusCode, subStatusCode) ||
statusCode == 0 // Gateway mode reports inability to connect due to
// PoolAcquirePendingLimitException as status code 0
// Upsert can return 404/0 in rare cases (when due to TTL expiration there is a race condition
FabianMeiswinkel marked this conversation as resolved.
Show resolved Hide resolved
case ItemWriteStrategy.ItemOverwrite =>
Exceptions.canBeTransientFailure(statusCode, subStatusCode) ||
statusCode == 0 || // Gateway mode reports inability to connect due to PoolAcquirePendingLimitException as status code 0
(statusCode == StatusCodes.NotFound && subStatusCode == 0)
FabianMeiswinkel marked this conversation as resolved.
Show resolved Hide resolved
case _ =>
Exceptions.canBeTransientFailure(statusCode, subStatusCode) ||
statusCode == 0 // Gateway mode reports inability to connect due to PoolAcquirePendingLimitException as status code 0
}
}

Expand Down Expand Up @@ -1273,6 +1290,20 @@ private object BulkWriter {
}
}

val emitFailureHandlerForComplete: EmitFailureHandler =
(signalType, emitResult) => {
if (emitResult.equals(EmitResult.FAIL_NON_SERIALIZED)) {
log.logDebug(s"emitFailureHandlerForComplete - Signal: ${signalType.toString}, Result: ${emitResult.toString}")
true
} else if (emitResult.equals(EmitResult.FAIL_CANCELLED) || emitResult.equals(EmitResult.FAIL_TERMINATED)) {
log.logDebug(s"emitFailureHandlerForComplete - Already completed - Signal: ${signalType.toString}, Result: ${emitResult.toString}")
FabianMeiswinkel marked this conversation as resolved.
Show resolved Hide resolved
false
} else {
log.logError(s"emitFailureHandlerForComplete - Signal: ${signalType.toString}, Result: ${emitResult.toString}")
false
}
}

private val bulkProcessingThresholds = new CosmosBulkExecutionThresholdsState()

// Custom bounded elastic scheduler to switch off IO thread to process response.
Expand Down
Loading
Loading