Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
The table of contents is too big for display.
Diff view
Diff view
  •  
  •  
  •  
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import cats.Show
import cats.data.{NonEmptyList, Validated, ValidatedNel}
import cats.implicits.*
import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
Copy link
Collaborator Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Overview of changes:

  • everywhere changed org.apache.logging.log4j.scala.Logging to tech.beshu.ror.utils.RequestIdAwareLogging
  • logger requires implicit instance of trait HasRequestId in scope
  • all implementations of trait RequestContext, BlockContext, RequestId, ResponseContext extend HasRequestId
  • in situations, where the request id is not available, the noRequestIdLogger is used

That way we known, that all logs either have the request id, or are using easily-searchable noRequestIdLogger

import org.json.JSONObject
import tech.beshu.ror.accesscontrol.audit.AuditingTool.AuditSettings.AuditSink
import tech.beshu.ror.accesscontrol.audit.AuditingTool.AuditSettings.AuditSink.{Disabled, Enabled}
Expand Down Expand Up @@ -148,7 +148,7 @@ final class AuditingTool private(auditSinks: NonEmptyList[BaseAuditSink])

}

object AuditingTool extends Logging {
object AuditingTool extends RequestIdAwareLogging {

final case class AuditSettings(auditSinks: NonEmptyList[AuditSettings.AuditSink])

Expand Down Expand Up @@ -212,10 +212,10 @@ object AuditingTool extends Logging {
createAuditSinks(settings, auditSinkServiceCreator).map {
_.map {
case Some(auditSinks) =>
logger.info(s"The audit is enabled with the given outputs: [${auditSinks.toList.show}]")
noRequestIdLogger.info(s"The audit is enabled with the given outputs: [${auditSinks.toList.show}]")
Some(new AuditingTool(auditSinks))
case None =>
logger.info("The audit is disabled because no output is enabled")
noRequestIdLogger.info("The audit is disabled because no output is enabled")
None
}
.toEither
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,10 +18,10 @@ package tech.beshu.ror.accesscontrol.audit.configurable

import cats.parse.{Parser0, Parser as P}
import cats.syntax.list.*
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.audit.utils.AuditSerializationHelper.AuditFieldValueDescriptor

object AuditFieldValueDescriptorParser extends Logging {
object AuditFieldValueDescriptorParser extends RequestIdAwareLogging {

private val lbrace = P.char('{')
private val rbrace = P.char('}')
Expand Down Expand Up @@ -62,7 +62,7 @@ object AuditFieldValueDescriptorParser extends Logging {
case "ECS_EVENT_OUTCOME" => Some(AuditFieldValueDescriptor.EcsEventOutcome)
case "REASON" => Some(AuditFieldValueDescriptor.Reason)
case "USER" =>
logger.warn(
noRequestIdLogger.warn(
"""The USER audit value placeholder is deprecated and should not be used in the configurable audit log serializer.
|Please use LOGGED_USER or PRESENTED_IDENTITY instead. Check the list of available placeholders in the documentation:
|https://docs.readonlyrest.com/elasticsearch/audit#using-configurable-serializer.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import cats.data.{NonEmptyList, Validated}
import cats.implicits.*
import eu.timepit.refined.types.string.NonEmptyString
import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.audit.sink.AuditDataStreamCreator.ErrorMessage
import tech.beshu.ror.accesscontrol.domain.{DataStreamName, RorAuditDataStream, TemplateName}
import tech.beshu.ror.es.DataStreamService
Expand All @@ -32,7 +32,7 @@ import tech.beshu.ror.utils.RefinedUtils.*

import java.util.concurrent.TimeUnit

final class AuditDataStreamCreator(services: NonEmptyList[DataStreamService]) extends Logging {
final class AuditDataStreamCreator(services: NonEmptyList[DataStreamService]) extends RequestIdAwareLogging {

def createIfNotExists(dataStreamName: RorAuditDataStream): Task[Either[NonEmptyList[ErrorMessage], Unit]] = {
services
Expand All @@ -47,7 +47,7 @@ final class AuditDataStreamCreator(services: NonEmptyList[DataStreamService]) ex
.checkDataStreamExists(dataStreamName.dataStream)
.flatMap {
case true =>
Task.delay(logger.info(s"Data stream ${dataStreamName.dataStream.show} already exists"))
Task.delay(noRequestIdLogger.info(s"Data stream ${dataStreamName.dataStream.show} already exists"))
.as(Valid(()))
case false =>
val settings = defaultSettingsFor(dataStreamName.dataStream)
Expand All @@ -57,19 +57,19 @@ final class AuditDataStreamCreator(services: NonEmptyList[DataStreamService]) ex

private def setupDataStream(service: DataStreamService, settings: DataStreamSettings): Task[Validated[ErrorMessage, Unit]] = {
for {
_ <- Task.delay(logger.info(s"Trying to setup ROR audit data stream ${settings.dataStreamName.show} with default settings.."))
_ <- Task.delay(noRequestIdLogger.info(s"Trying to setup ROR audit data stream ${settings.dataStreamName.show} with default settings.."))
result <- service.fullySetupDataStream(settings).attempt
finalResult <- result match {
case Right(DataStreamSetupResult.Success) =>
Task.delay(logger.info(s"ROR audit data stream ${settings.dataStreamName.show} created."))
Task.delay(noRequestIdLogger.info(s"ROR audit data stream ${settings.dataStreamName.show} created."))
.as(Valid(()))
case Right(DataStreamSetupResult.Failure(reason)) =>
val message = s"Failed to setup ROR audit data stream ${settings.dataStreamName.show}. Reason: ${reason.show}"
Task.delay(logger.error(message))
Task.delay(noRequestIdLogger.error(message))
.as(ErrorMessage(message).invalid)
case Left(ex) =>
val message = s"An unexpected error occurred while setting up the ROR audit data stream ${settings.dataStreamName.show}. Details: ${ex.getMessage}"
Task.delay(logger.error(message, ex))
Task.delay(noRequestIdLogger.error(message, ex))
.as(ErrorMessage(message).invalid)
}
} yield finalResult
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package tech.beshu.ror.accesscontrol.blocks
import cats.data.{NonEmptyList, Validated, WriterT}
import cats.{Eq, Show}
import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.audit.LoggingContext
import tech.beshu.ror.accesscontrol.blocks.Block.*
import tech.beshu.ror.accesscontrol.blocks.Block.ExecutionResult.{Matched, Mismatched}
Expand All @@ -43,7 +43,7 @@ class Block(val name: Name,
val audit: Audit,
val rules: NonEmptyList[Rule])
(implicit val loggingContext: LoggingContext)
extends Logging {
extends RequestIdAwareLogging {

import Lifter.*

Expand Down Expand Up @@ -73,7 +73,7 @@ class Block(val name: Name,
val ruleResult = rule
.check[B](blockContext)
.recover { case e =>
logger.error(s"[${blockContext.requestContext.id.show}] ${name.show}: ${rule.name.show} rule matching got an error ${e.getMessage}", e)
logger.error(s"${name.show}: ${rule.name.show} rule matching got an error ${e.getMessage}", e)(blockContext)
RuleResult.Rejected[B]()
}
lift[B](ruleResult)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,9 @@ import tech.beshu.ror.accesscontrol.domain.GroupsLogic.{Combined, NegativeGroups
import tech.beshu.ror.accesscontrol.request.RequestContext
import tech.beshu.ror.syntax.*

sealed trait BlockContext {
sealed trait BlockContext extends HasRequestId {
override def requestId: RequestId = requestContext.id.toRequestId

def requestContext: RequestContext

def userMetadata: UserMetadata
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package tech.beshu.ror.accesscontrol.blocks

import cats.Order
import cats.implicits.*
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.blocks.rules.Rule
import tech.beshu.ror.accesscontrol.blocks.rules.auth.*
import tech.beshu.ror.accesscontrol.blocks.rules.elasticsearch.*
Expand All @@ -29,20 +29,20 @@ import tech.beshu.ror.accesscontrol.blocks.rules.tranport.*
import tech.beshu.ror.accesscontrol.domain.GroupsLogic.*
import tech.beshu.ror.accesscontrol.orders.*

class RuleOrdering extends Ordering[Rule] with Logging {
class RuleOrdering extends Ordering[Rule] with RequestIdAwareLogging {

override def compare(rule1: Rule, rule2: Rule): Int = {
val rule1TypeIndex = RuleOrdering.orderedListOrRuleType.indexOf(rule1.getClass)
val rule2TypeIndex = RuleOrdering.orderedListOrRuleType.indexOf(rule2.getClass)
(rule1TypeIndex, rule2TypeIndex) match {
case (-1, -1) =>
logger.warn(s"No order defined for rules: ${rule1.name.show}, ${rule1.name.show}")
noRequestIdLogger.warn(s"No order defined for rules: ${rule1.name.show}, ${rule1.name.show}")
implicitly[Order[Rule.Name]].compare(rule1.name, rule2.name)
case (-1, _) =>
logger.warn(s"No order defined for rule: ${rule1.name.show}")
noRequestIdLogger.warn(s"No order defined for rule: ${rule1.name.show}")
1
case (_, -1) =>
logger.warn(s"No order defined for rule: ${rule2.name.show}")
noRequestIdLogger.warn(s"No order defined for rule: ${rule2.name.show}")
-1
case (i1, i2) =>
i1 compareTo i2
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import eu.timepit.refined.auto.*
import eu.timepit.refined.types.string.NonEmptyString
import io.lemonlabs.uri.Url
import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.blocks.definitions.ExternalAuthorizationService.Name
import tech.beshu.ror.accesscontrol.blocks.definitions.HttpExternalAuthorizationService.*
import tech.beshu.ror.accesscontrol.blocks.definitions.HttpExternalAuthorizationService.Config.*
Expand Down Expand Up @@ -62,7 +62,7 @@ final class HttpExternalAuthorizationService(override val id: ExternalAuthorizat
val config: HttpExternalAuthorizationService.Config,
httpClient: HttpClient)
extends ExternalAuthorizationService
with Logging {
with RequestIdAwareLogging {

override def grantsFor(userId: User.Id)
(implicit requestId: RequestId): Task[UniqueList[Group]] = {
Expand Down Expand Up @@ -106,10 +106,10 @@ final class HttpExternalAuthorizationService(override val id: ExternalAuthorizat
val groupsFromBody = groupsFrom(body)
groupsFromBody match {
case Success(groups) =>
logger.debug(s"[${requestId.show}] Groups returned by groups provider '${id.show}': ${groups.show}")
logger.debug(s"Groups returned by groups provider '${id.show}': ${groups.show}")
UniqueList.from(groups)
case Failure(ex) =>
logger.debug(s"[${requestId.show}] Group based authorization response exception - provider '${id.show}'", ex)
logger.debug(s"Group based authorization response exception - provider '${id.show}'", ex)
UniqueList.empty
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import cats.Show
import cats.implicits.toShow
import monix.catnap.*
import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.blocks.definitions.CircuitBreakerConfig
import tech.beshu.ror.accesscontrol.domain
import tech.beshu.ror.accesscontrol.domain.{Group, GroupIdLike, RequestId, User}
Expand Down Expand Up @@ -113,7 +113,7 @@ class CircuitBreakerLdapUsersServiceDecorator(val underlying: LdapUsersService,
override def serviceTimeout: PositiveFiniteDuration = underlying.serviceTimeout
}

trait LdapCircuitBreaker extends Logging {
trait LdapCircuitBreaker extends RequestIdAwareLogging {
this: LdapService =>

protected def circuitBreakerConfig: CircuitBreakerConfig
Expand All @@ -126,16 +126,16 @@ trait LdapCircuitBreaker extends Logging {
maxFailures = maxFailures.value,
resetTimeout = resetDuration.value,
onRejected = Task {
logger.debug(s"LDAP ${id.show} circuit breaker rejected task (Open or HalfOpen state)")
noRequestIdLogger.debug(s"LDAP ${id.show} circuit breaker rejected task (Open or HalfOpen state)")
},
onClosed = Task {
logger.debug(s"LDAP ${id.show} circuit breaker is accepting tasks again (switched to Close state)")
noRequestIdLogger.debug(s"LDAP ${id.show} circuit breaker is accepting tasks again (switched to Close state)")
},
onHalfOpen = Task {
logger.debug(s"LDAP ${id.show} circuit breaker accepted one task for testing (switched to HalfOpen state)")
noRequestIdLogger.debug(s"LDAP ${id.show} circuit breaker accepted one task for testing (switched to HalfOpen state)")
},
onOpen = Task {
logger.debug(s"LDAP ${id.show} circuit breaker rejected task (switched to Open state)")
noRequestIdLogger.debug(s"LDAP ${id.show} circuit breaker rejected task (switched to Open state)")
}
)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
package tech.beshu.ror.accesscontrol.blocks.definitions.ldap

import monix.eval.Task
import org.apache.logging.log4j.scala.Logging
import tech.beshu.ror.utils.RequestIdAwareLogging
import tech.beshu.ror.accesscontrol.domain
import tech.beshu.ror.accesscontrol.domain.{Group, GroupIdLike, RequestId, User}
import tech.beshu.ror.implicits.*
Expand All @@ -29,17 +29,17 @@ import scala.util.{Failure, Success}

class LoggableLdapAuthenticationServiceDecorator(val underlying: LdapAuthenticationService)
extends LdapAuthenticationService
with Logging {
with RequestIdAwareLogging {

override def authenticate(user: User.Id, secret: domain.PlainTextSecret)(implicit requestId: RequestId): Task[Boolean] = {
logger.debug(s"[${requestId.show}] Trying to authenticate user [${user.show}] with LDAP [${id.show}]")
logger.debug(s"Trying to authenticate user [${user.show}] with LDAP [${id.show}]")
underlying
.authenticate(user, secret)
.andThen {
case Success(authenticationResult) =>
logger.debug(s"[${requestId.show}] User [${user.show}]${if (authenticationResult) "" else " not"} authenticated by LDAP [${id.show}]")
logger.debug(s"User [${user.show}]${if (authenticationResult) "" else " not"} authenticated by LDAP [${id.show}]")
case Failure(ex) =>
logger.debug(s"[${requestId.show}] LDAP authentication failed:", ex)
logger.debug(s"LDAP authentication failed:", ex)
}
}

Expand All @@ -63,18 +63,18 @@ object LoggableLdapAuthorizationService {

class WithoutGroupsFilteringDecorator(val underlying: LdapAuthorizationService.WithoutGroupsFiltering)
extends LdapAuthorizationService.WithoutGroupsFiltering
with Logging {
with RequestIdAwareLogging {

override def groupsOf(userId: User.Id)
(implicit requestId: RequestId): Task[UniqueList[Group]] = {
logger.debug(s"[${requestId.show}] Trying to fetch user [id=${userId.show}] groups from LDAP [${id.show}]")
logger.debug(s"Trying to fetch user [id=${userId.show}] groups from LDAP [${id.show}]")
underlying
.groupsOf(userId)
.andThen {
case Success(groups) =>
logger.debug(s"[${requestId.show}] LDAP [${id.show}] returned for user [${userId.show}] following groups: [${groups.show}]")
logger.debug(s"LDAP [${id.show}] returned for user [${userId.show}] following groups: [${groups.show}]")
case Failure(ex) =>
logger.debug(s"[${requestId.show}] Fetching LDAP user's groups failed:", ex)
logger.debug(s"Fetching LDAP user's groups failed:", ex)
}
}

Expand All @@ -87,18 +87,18 @@ object LoggableLdapAuthorizationService {

class WithGroupsFilteringDecorator(val underlying: LdapAuthorizationService.WithGroupsFiltering)
extends LdapAuthorizationService.WithGroupsFiltering
with Logging {
with RequestIdAwareLogging {

override def groupsOf(userId: User.Id, filteringGroupIds: Set[GroupIdLike])
(implicit requestId: RequestId): Task[UniqueList[Group]] = {
logger.debug(s"[${requestId.show}] Trying to fetch user [id=${userId.show}] groups from LDAP [${id.show}] (assuming that filtered group IDs are [${filteringGroupIds.show}])")
logger.debug(s"Trying to fetch user [id=${userId.show}] groups from LDAP [${id.show}] (assuming that filtered group IDs are [${filteringGroupIds.show}])")
underlying
.groupsOf(userId, filteringGroupIds)
.andThen {
case Success(groups) =>
logger.debug(s"[${requestId.show}] LDAP [${id.show}] returned for user [${userId.show}] following groups: [${groups.show}]")
logger.debug(s"LDAP [${id.show}] returned for user [${userId.show}] following groups: [${groups.show}]")
case Failure(ex) =>
logger.debug(s"[${requestId.show}] Fetching LDAP user's groups failed:", ex)
logger.debug(s"Fetching LDAP user's groups failed:", ex)
}
}

Expand All @@ -113,20 +113,20 @@ object LoggableLdapAuthorizationService {

private class LoggableLdapUsersServiceDecorator(underlying: LdapUsersService)
extends LdapUsersService
with Logging {
with RequestIdAwareLogging {

override def ldapUserBy(userId: User.Id)(implicit requestId: RequestId): Task[Option[LdapUser]] = {
logger.debug(s"[${requestId.show}] Trying to fetch user with identifier [${userId.show}] from LDAP [${id.show}]")
logger.debug(s"Trying to fetch user with identifier [${userId.show}] from LDAP [${id.show}]")
underlying
.ldapUserBy(userId)
.andThen {
case Success(ldapUser) =>
ldapUser match {
case Some(user) => logger.debug(s"[${requestId.show}] User with identifier [${userId.show}] found [dn = ${user.dn.show}]")
case None => logger.debug(s"[${requestId.show}] User with identifier [${userId.show}] not found")
case Some(user) => logger.debug(s"User with identifier [${userId.show}] found [dn = ${user.dn.show}]")
case None => logger.debug(s"User with identifier [${userId.show}] not found")
}
case Failure(ex) =>
logger.debug(s"[${requestId.show}] Fetching LDAP user failed:", ex)
logger.debug(s"Fetching LDAP user failed:", ex)
}
}

Expand Down
Loading
Loading