diff options
22 files changed, 642 insertions, 293 deletions
diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HeaderValidator.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HeaderValidator.kt new file mode 100644 index 00000000..9d8accae --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/HeaderValidator.kt @@ -0,0 +1,60 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 NOKIA + * ================================================================================ + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ============LICENSE_END========================================================= + */ + +package org.onap.dcae.collectors.veshv.impl + +import arrow.core.Either +import arrow.data.Nel +import arrow.data.NonEmptyList +import com.google.protobuf.Descriptors +import org.onap.dcae.collectors.veshv.domain.headerRequiredFieldDescriptors +import org.onap.dcae.collectors.veshv.domain.vesEventListenerVersionRegex +import org.onap.ves.VesEventOuterClass.CommonEventHeader + +typealias Validator = (CommonEventHeader) -> List<ValidationError> + +object HeaderValidator { + private val validators = (listOf(validateEventListenerVersion()) + + headerRequiredFieldDescriptors.map { fieldDescriptor -> + validateRequiredField(fieldDescriptor) + }) + + + fun validate(header: CommonEventHeader): Either<Nel<ValidationError>, CommonEventHeader> { + val result: List<ValidationError> = validators.flatMap { it(header) } + + return Either.cond(result.isEmpty(), { header }, { NonEmptyList.fromListUnsafe(result) }) + } + + private fun validateEventListenerVersion(): Validator = { header: CommonEventHeader -> + if (!vesEventListenerVersionRegex.matches(header.vesEventListenerVersion)) + listOf(ValidationError.VersionMismatch(header.vesEventListenerVersion)) + else + emptyList() + } + + private fun validateRequiredField(requiredField: Descriptors.FieldDescriptor): Validator = + { header: CommonEventHeader -> + if (!header.hasField(requiredField)) + listOf(ValidationError.MissingField(requiredField.name)) + else + emptyList() + } +} diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt index 93940752..66d2ea0c 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidator.kt @@ -20,11 +20,9 @@ package org.onap.dcae.collectors.veshv.impl import arrow.core.Either +import arrow.data.Nel import org.onap.dcae.collectors.veshv.domain.WireFrameMessage -import org.onap.dcae.collectors.veshv.domain.headerRequiredFieldDescriptors -import org.onap.dcae.collectors.veshv.domain.vesEventListenerVersionRegex import org.onap.dcae.collectors.veshv.model.VesMessage -import org.onap.ves.VesEventOuterClass.CommonEventHeader typealias ValidationFailMessage = () -> String typealias ValidationSuccessMessage = () -> String @@ -33,24 +31,21 @@ typealias ValidationResult = Either<ValidationFailMessage, ValidationSuccessMess internal object MessageValidator { fun validateFrameMessage(message: WireFrameMessage): ValidationResult = - message.validate().fold({ - Either.left { "Invalid wire frame header, reason: ${it.message}" } - }, { - Either.right { "Wire frame header is valid" } - }) + message.validate().fold({ + Either.left { "Invalid wire frame header, reason: ${it.message}" } + }, { + Either.right { "Wire frame header is valid" } + }) fun validateProtobufMessage(message: VesMessage): ValidationResult = - if (message.isValid()) { + HeaderValidator.validate(message.header).fold( + { validationErrors: Nel<ValidationError> -> + Either.left { + "Protocol buffer message is invalid, reasons:" + validationErrors.all + .joinToString(prefix = "\n-") { it.errorMessage } + } + }, + { Either.right { "Protocol buffers message is valid" } - } else { - Either.left { "Unsupported protocol buffers message." } - } - - fun VesMessage.isValid() = allMandatoryFieldsArePresent(this.header) - .and(vesEventListenerVersionRegex.matches(header.vesEventListenerVersion)) - - private fun allMandatoryFieldsArePresent(header: CommonEventHeader) = - headerRequiredFieldDescriptors - .all { fieldDescriptor -> header.hasField(fieldDescriptor) } - + }) } diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt index 0977595a..6105b585 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/Router.kt @@ -21,7 +21,7 @@ package org.onap.dcae.collectors.veshv.impl import arrow.core.Option import org.onap.dcae.collectors.veshv.model.ClientContext -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.debug +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.debug import org.onap.dcae.collectors.veshv.model.RoutedMessage import org.onap.dcae.collectors.veshv.model.Routing import org.onap.dcae.collectors.veshv.model.VesMessage diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/ValidationError.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/ValidationError.kt new file mode 100644 index 00000000..56a77f4f --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/ValidationError.kt @@ -0,0 +1,34 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 NOKIA + * ================================================================================ + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ============LICENSE_END========================================================= + */ +package org.onap.dcae.collectors.veshv.impl + +import org.onap.dcae.collectors.veshv.domain.vesEventListenerVersionRegex + +sealed class ValidationError(val errorMessage: String) { + class MissingField<A>(field: A) : ValidationError( + "Invalid header - missing $field field" + ) + + class VersionMismatch(actualVersion: String) : ValidationError( + "Invalid header - vesEventListenerVersion mismatch. " + + "Expected $vesEventListenerVersionRegex, but was $actualVersion" + ) +} + diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt index 0d07504d..cf73aed8 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/VesHvCollector.kt @@ -26,6 +26,7 @@ import org.onap.dcae.collectors.veshv.boundary.Metrics import org.onap.dcae.collectors.veshv.boundary.Sink import org.onap.dcae.collectors.veshv.domain.ByteData import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.handleReactiveStreamError import org.onap.dcae.collectors.veshv.impl.wire.WireChunkDecoder import org.onap.dcae.collectors.veshv.model.ClientContext import org.onap.dcae.collectors.veshv.model.RoutedMessage @@ -33,7 +34,6 @@ import org.onap.dcae.collectors.veshv.model.VesMessage import org.onap.dcae.collectors.veshv.utils.logging.Logger import org.onap.dcae.collectors.veshv.utils.logging.filterEmptyWithLog import org.onap.dcae.collectors.veshv.utils.logging.filterFailedWithLog -import org.onap.dcae.collectors.veshv.utils.logging.handleReactiveStreamError import reactor.core.publisher.Flux import reactor.core.publisher.Mono @@ -56,7 +56,7 @@ internal class VesHvCollector( .transform(::decodeProtobufPayload) .transform(::filterInvalidProtobufMessages) .transform(::routeMessage) - .onErrorResume { logger.handleReactiveStreamError(clientContext::asMap, it) } + .onErrorResume { logger.handleReactiveStreamError(clientContext, it) } .doFinally { releaseBuffersMemory() } .then() diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ClientContextLogging.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ClientContextLogging.kt new file mode 100644 index 00000000..21b79bbe --- /dev/null +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/ClientContextLogging.kt @@ -0,0 +1,47 @@ +/* + * ============LICENSE_START======================================================= + * dcaegen2-collectors-veshv + * ================================================================================ + * Copyright (C) 2018 NOKIA + * ================================================================================ + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ============LICENSE_END========================================================= + */ +package org.onap.dcae.collectors.veshv.impl.adapters + +import org.onap.dcae.collectors.veshv.model.ClientContext +import org.onap.dcae.collectors.veshv.utils.logging.AtLevelLogger +import org.onap.dcae.collectors.veshv.utils.logging.Logger +import org.onap.dcae.collectors.veshv.utils.logging.handleReactiveStreamError +import reactor.core.publisher.Flux + +@Suppress("TooManyFunctions") +internal object ClientContextLogging { + fun Logger.withError(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withError(ctx::asMap, block) + fun Logger.withWarn(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withWarn(ctx::asMap, block) + fun Logger.withInfo(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withInfo(ctx::asMap, block) + fun Logger.withDebug(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withDebug(ctx::asMap, block) + fun Logger.withTrace(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withTrace(ctx::asMap, block) + + fun Logger.error(ctx: ClientContext, message: () -> String) = error(ctx::asMap, message) + fun Logger.warn(ctx: ClientContext, message: () -> String) = warn(ctx::asMap, message) + fun Logger.info(ctx: ClientContext, message: () -> String) = info(ctx::asMap, message) + fun Logger.debug(ctx: ClientContext, message: () -> String) = debug(ctx::asMap, message) + fun Logger.trace(ctx: ClientContext, message: () -> String) = trace(ctx::asMap, message) + + fun <T> Logger.handleReactiveStreamError(context: ClientContext, ex: Throwable, + returnFlux: Flux<T> = Flux.empty()): Flux<T> { + return this.handleReactiveStreamError({ context.asMap() }, ex, returnFlux) + } +} + diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt index f6cb018f..ec8593af 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/LoggingSinkProvider.kt @@ -22,8 +22,8 @@ package org.onap.dcae.collectors.veshv.impl.adapters import org.onap.dcae.collectors.veshv.boundary.Sink import org.onap.dcae.collectors.veshv.boundary.SinkProvider import org.onap.dcae.collectors.veshv.model.ClientContext -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.info -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.trace +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.info +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.trace import org.onap.dcae.collectors.veshv.model.CollectorConfiguration import org.onap.dcae.collectors.veshv.model.RoutedMessage import org.onap.dcae.collectors.veshv.utils.logging.Logger diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt index 07ce7604..690a7d1e 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/adapters/kafka/KafkaSink.kt @@ -21,7 +21,7 @@ package org.onap.dcae.collectors.veshv.impl.adapters.kafka import org.onap.dcae.collectors.veshv.boundary.Sink import org.onap.dcae.collectors.veshv.model.ClientContext -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.withWarn +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.withWarn import org.onap.dcae.collectors.veshv.utils.logging.Marker import org.onap.dcae.collectors.veshv.model.RoutedMessage import org.onap.dcae.collectors.veshv.model.VesMessage diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt index 3fa05c4d..6f02d43e 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/socket/NettyTcpServer.kt @@ -24,9 +24,9 @@ import arrow.effects.IO import org.onap.dcae.collectors.veshv.boundary.CollectorProvider import org.onap.dcae.collectors.veshv.boundary.Server import org.onap.dcae.collectors.veshv.model.ClientContext -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.info -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.debug -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.withWarn +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.info +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.debug +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.withWarn import org.onap.dcae.collectors.veshv.utils.logging.Marker import org.onap.dcae.collectors.veshv.model.ServerConfiguration import org.onap.dcae.collectors.veshv.ssl.boundary.ServerSslContextFactory diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt index 349b0787..b735138d 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/impl/wire/WireChunkDecoder.kt @@ -27,10 +27,11 @@ import org.onap.dcae.collectors.veshv.domain.WireFrameDecoder import org.onap.dcae.collectors.veshv.domain.WireFrameDecodingError import org.onap.dcae.collectors.veshv.domain.WireFrameMessage import org.onap.dcae.collectors.veshv.model.ClientContext -import org.onap.dcae.collectors.veshv.model.ClientContextLogging.trace import org.onap.dcae.collectors.veshv.utils.logging.Logger -import org.onap.dcae.collectors.veshv.utils.logging.handleReactiveStreamError +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.handleReactiveStreamError +import org.onap.dcae.collectors.veshv.impl.adapters.ClientContextLogging.trace import reactor.core.publisher.Flux +import reactor.core.publisher.Flux.defer import reactor.core.publisher.SynchronousSink /** @@ -46,7 +47,7 @@ internal class WireChunkDecoder( streamBuffer.release() } - fun decode(byteBuf: ByteBuf): Flux<WireFrameMessage> = Flux.defer { + fun decode(byteBuf: ByteBuf): Flux<WireFrameMessage> = defer { logIncomingMessage(byteBuf) if (byteBuf.readableBytes() == 0) { byteBuf.release() @@ -54,7 +55,7 @@ internal class WireChunkDecoder( } else { streamBuffer.addComponent(true, byteBuf) generateFrames() - .onErrorResume { logger.handleReactiveStreamError(ctx::asMap, it, Flux.error(it)) } + .onErrorResume { logger.handleReactiveStreamError(ctx, it, Flux.error(it)) } .doFinally { streamBuffer.discardReadComponents() } } } diff --git a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ClientContext.kt b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ClientContext.kt index 213b7434..305e4cb1 100644 --- a/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ClientContext.kt +++ b/sources/hv-collector-core/src/main/kotlin/org/onap/dcae/collectors/veshv/model/ClientContext.kt @@ -41,17 +41,3 @@ data class ClientContext( return result } } - -object ClientContextLogging { - fun Logger.withError(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withError(ctx::asMap, block) - fun Logger.withWarn(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withWarn(ctx::asMap, block) - fun Logger.withInfo(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withInfo(ctx::asMap, block) - fun Logger.withDebug(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withDebug(ctx::asMap, block) - fun Logger.withTrace(ctx: ClientContext, block: AtLevelLogger.() -> Unit) = withTrace(ctx::asMap, block) - - fun Logger.error(ctx: ClientContext, message: () -> String) = error(ctx::asMap, message) - fun Logger.warn(ctx: ClientContext, message: () -> String) = warn(ctx::asMap, message) - fun Logger.info(ctx: ClientContext, message: () -> String) = info(ctx::asMap, message) - fun Logger.debug(ctx: ClientContext, message: () -> String) = debug(ctx::asMap, message) - fun Logger.trace(ctx: ClientContext, message: () -> String) = trace(ctx::asMap, message) -} diff --git a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt index 60bd767b..f784daa4 100644 --- a/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt +++ b/sources/hv-collector-core/src/test/kotlin/org/onap/dcae/collectors/veshv/impl/MessageValidatorTest.kt @@ -25,19 +25,14 @@ import com.nhaarman.mockitokotlin2.doReturn import com.nhaarman.mockitokotlin2.mock import org.assertj.core.api.Assertions.assertThat import org.jetbrains.spek.api.Spek -import org.jetbrains.spek.api.dsl.describe -import org.jetbrains.spek.api.dsl.given -import org.jetbrains.spek.api.dsl.it -import org.jetbrains.spek.api.dsl.on -import org.onap.dcae.collectors.veshv.domain.ByteData -import org.onap.dcae.collectors.veshv.domain.InvalidMajorVersion -import org.onap.dcae.collectors.veshv.domain.VesEventDomain -import org.onap.dcae.collectors.veshv.domain.WireFrameMessage +import org.jetbrains.spek.api.dsl.* +import org.onap.dcae.collectors.veshv.domain.* import org.onap.dcae.collectors.veshv.model.VesMessage import org.onap.dcae.collectors.veshv.tests.utils.commonHeader import org.onap.dcae.collectors.veshv.tests.utils.vesEventBytes import org.onap.ves.VesEventOuterClass.CommonEventHeader.* import kotlin.test.assertTrue +import kotlin.test.fail internal object MessageValidatorTest : Spek({ @@ -50,7 +45,8 @@ internal object MessageValidatorTest : Spek({ it("should accept message with fully initialized message header") { val vesMessage = VesMessage(commonHeader, vesEventBytes(commonHeader)) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isTrue() + assertThat(validateProtobufMessage(vesMessage).isRight()) + .describedAs("message validation result").isTrue() } } @@ -59,7 +55,8 @@ internal object MessageValidatorTest : Spek({ val header = commonHeader(domain) val vesMessage = VesMessage(header, vesEventBytes(header)) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isTrue() + assertThat(validateProtobufMessage(vesMessage).isRight()) + .describedAs("message validation result").isTrue() } } } @@ -68,46 +65,83 @@ internal object MessageValidatorTest : Spek({ on("ves hv message bytes") { val vesMessage = VesMessage(getDefaultInstance(), ByteData.EMPTY) it("should not accept message with default header") { + with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isFalse() + validateProtobufMessage(vesMessage).fold({ + val failMessages = it.invoke() + + val containsAllErrorMessages = failMessages.contains("vesEventListenerVersion mismatch") + && failMessages.contains("missing domain field") + && failMessages.contains("missing version field") + && failMessages.contains("missing priority field") + && failMessages.contains("missing eventId field") + && failMessages.contains("missing eventName field") + && failMessages.contains("missing lastEpochMicrosec field") + && failMessages.contains("missing startEpochMicrosec field") + && failMessages.contains("missing reportingEntityName field") + && failMessages.contains("missing sourceName field") + && failMessages.contains("missing vesEventListenerVersion field") + + assertThat(containsAllErrorMessages) + .describedAs("message validation result").isTrue() + }, { + fail() + }) } } } - val priorityTestCases = mapOf( + given("priority test cases") { + mapOf( Priority.PRIORITY_NOT_PROVIDED to false, Priority.LOW to true, Priority.MEDIUM to true, Priority.HIGH to true - ) + ).forEach { value, expectedResult -> + on("ves hv message including header with priority $value") { + val commonEventHeader = commonHeader(priority = value) + val vesMessage = VesMessage(commonEventHeader, vesEventBytes(commonEventHeader)) - priorityTestCases.forEach { value, expectedResult -> - on("ves hv message including header with priority $value") { - val commonEventHeader = commonHeader(priority = value) - val vesMessage = VesMessage(commonEventHeader, vesEventBytes(commonEventHeader)) - - it("should resolve validation result") { - with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation results") + it("should resolve validation result") { + with(cut) { + assertThat(validateProtobufMessage(vesMessage).isRight()) + .describedAs("message validation results") .isEqualTo(expectedResult) + } } } } } + on("ves hv message including header with not initialized fields") { val commonHeader = newBuilder() - .setVersion("1.9") - .setEventName("Sample event name") - .setEventId("Sample event Id") - .setSourceName("Sample Source") - .build() + .setVersion("1.9") + .setEventName("Sample event name") + .setEventId("Sample event Id") + .setSourceName("Sample Source") + .build() val rawMessageBytes = vesEventBytes(commonHeader) it("should not accept not fully initialized message header") { val vesMessage = VesMessage(commonHeader, rawMessageBytes) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isFalse() + validateProtobufMessage(vesMessage).fold({ + val failMessages = it.invoke() + + val containsAllErrorMessages = failMessages.contains("vesEventListenerVersion mismatch") + && failMessages.contains("missing domain field") + && failMessages.contains("missing priority field") + && failMessages.contains("missing lastEpochMicrosec field") + && failMessages.contains("missing startEpochMicrosec field") + && failMessages.contains("missing reportingEntityName field") + && failMessages.contains("missing vesEventListenerVersion field") + + assertThat(containsAllErrorMessages).describedAs("message validation result") + .isTrue() + }, { + fail() + }) } } } @@ -120,7 +154,15 @@ internal object MessageValidatorTest : Spek({ it("should not accept message header") { val vesMessage = VesMessage(commonHeader, rawMessageBytes) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isFalse() + validateProtobufMessage(vesMessage).fold({ + val failMessages = it.invoke() + + assertThat(failMessages.contains("vesEventListenerVersion mismatch")) + .describedAs("message validation result") + .isTrue() + }, { + fail() + }) } } } @@ -133,7 +175,15 @@ internal object MessageValidatorTest : Spek({ val vesMessage = VesMessage(commonHeader, rawMessageBytes) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isFalse() + validateProtobufMessage(vesMessage).fold({ + val failMessages = it.invoke() + + assertThat(failMessages.contains("vesEventListenerVersion mismatch")) + .describedAs("message validation result") + .isTrue() + }, { + fail() + }) } } } @@ -146,7 +196,15 @@ internal object MessageValidatorTest : Spek({ val vesMessage = VesMessage(commonHeader, rawMessageBytes) with(cut) { - assertThat(vesMessage.isValid()).describedAs("message validation result").isFalse() + validateProtobufMessage(vesMessage).fold({ + val failMessages = it.invoke() + + assertThat(failMessages.contains("vesEventListenerVersion mismatch")) + .describedAs("message validation result") + .isTrue() + }, { + fail() + }) } } } diff --git a/sources/hv-collector-core/src/test/resources/logback-test.xml b/sources/hv-collector-core/src/test/resources/logback-test.xml index 9a4eacfe..06aa3adc 100644 --- a/sources/hv-collector-core/src/test/resources/logback-test.xml +++ b/sources/hv-collector-core/src/test/resources/logback-test.xml @@ -1,35 +1,43 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg%n"/> - <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> - <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> - </encoder> - </appender> - - <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>${SIMPLE_LOG_PATTERN}</pattern> + </encoder> </appender> - <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> + <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> - <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> + <root level="INFO"> + <appender-ref ref="CONSOLE"/> </root> </configuration> diff --git a/sources/hv-collector-ct/src/test/resources/logback-test.xml b/sources/hv-collector-ct/src/test/resources/logback-test.xml index 93f22771..40a20fc1 100644 --- a/sources/hv-collector-ct/src/test/resources/logback-test.xml +++ b/sources/hv-collector-ct/src/test/resources/logback-test.xml @@ -1,35 +1,64 @@ <?xml version="1.0" encoding="UTF-8"?> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> - - <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> - <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> - </encoder> - </appender> + <property name="LOG_FILE" + value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> + + + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_thr" value="%thread"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="p_mdc" value="%replace(%replace(%mdc){'\t', '\\\\t'}){'\n', '\\\\n'}"/> + <property name="p_msg" value="%replace(%replace(%msg){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_exc" value="%replace(%replace(%rootException){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_mak" value="%replace(%replace(%marker){'\t', '\\\\t'}){'\n','\\\\n'}"/> + + <property name="READABLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| ${p_mak}\t +| %rootException\t +| ${p_mdc}\t +| ${p_thr}%n"/> + + <property name="ONAP_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_thr}\t +| ${p_lvl}\t +| ${p_log}\t +| ${p_mdc}\t +| ${p_msg}\t +| ${p_exc}\t +| ${p_mak}%n"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>${READABLE_LOG_PATTERN}</pattern> + </encoder> + </appender> <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${ONAP_LOG_PATTERN}</pattern> + </encoder> + <file>${LOG_FILE}</file> + <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> + <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> </appender> - <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> - <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> </root> </configuration> diff --git a/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml b/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml index 4d12b113..ba07c9c4 100644 --- a/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml +++ b/sources/hv-collector-dcae-app-simulator/src/main/resources/logback.xml @@ -1,36 +1,95 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + <property name="COMPONENT_NAME" + value="hv-ves-dcae-app-simulator"/> + <property name="COMPONENT_SHORT_NAME" + value="dcae-app-simulator"/> - <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> - <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> - </encoder> - </appender> + <property name="LOG_FILENAME" value="${COMPONENT_SHORT_NAME}"/> + <property name="LOG_PATH" value="/var/log/ONAP/${COMPONENT_NAME}"/> + <property name="ARCHIVE" value="${LOG_PATH}/archive"/> + + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_thr" value="%thread"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="p_mdc" value="%replace(%replace(%mdc){'\t', '\\\\t'}){'\n', '\\\\n'}"/> + <property name="p_msg" value="%replace(%replace(%msg){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_exc" value="%replace(%replace(%rootException){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_mak" value="%replace(%replace(%marker){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| %rootException%n"/> + <property name="READABLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| ${p_mak}\t +| %rootException\t +| ${p_mdc}\t +| ${p_thr}%n"/> + <property name="ONAP_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_thr}\t +| ${p_lvl}\t +| ${p_log}\t +| ${p_mdc}\t +| ${p_msg}\t +| ${p_exc}\t +| ${p_mak}%n"/> + <property name="LOG_PATTERN_IN_USE" value="${SIMPLE_LOG_PATTERN}"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>${LOG_PATTERN_IN_USE}</pattern> + </encoder> + </appender> <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${LOG_PATTERN_IN_USE}</pattern> + </encoder> + <file>${LOG_PATH}/${LOG_FILENAME}.log</file> + <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy"> + <FileNamePattern>${ARCHIVE}/${LOG_FILENAME}.%d{yyyy-MM-dd}.%i.log.gz</FileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> </appender> - <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> - <!--<logger name="reactor.netty" level="DEBUG"/>--> + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + <!--<logger name="reactor.netty" level="DEBUG"/>--> - <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> </root> </configuration>
\ No newline at end of file diff --git a/sources/hv-collector-domain/src/test/resources/logback.xml b/sources/hv-collector-domain/src/test/resources/logback.xml deleted file mode 100644 index 0bf2cb02..00000000 --- a/sources/hv-collector-domain/src/test/resources/logback.xml +++ /dev/null @@ -1,54 +0,0 @@ -<?xml version="1.0" encoding="UTF-8"?> -<!-- - ~ ============LICENSE_START======================================================= - ~ dcaegen2-collectors-veshv - ~ ================================================================================ - ~ Copyright (C) 2018 NOKIA - ~ ================================================================================ - ~ Licensed under the Apache License, Version 2.0 (the "License"); - ~ you may not use this file except in compliance with the License. - ~ You may obtain a copy of the License at - ~ - ~ http://www.apache.org/licenses/LICENSE-2.0 - ~ - ~ Unless required by applicable law or agreed to in writing, software - ~ distributed under the License is distributed on an "AS IS" BASIS, - ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - ~ See the License for the specific language governing permissions and - ~ limitations under the License. - ~ ============LICENSE_END========================================================= - --> -<configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> - - <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> - <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> - </encoder> - </appender> - - <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> - </appender> - - <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> - - <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> - </root> -</configuration>
\ No newline at end of file diff --git a/sources/hv-collector-health-check/pom.xml b/sources/hv-collector-health-check/pom.xml index 3e5c6aa0..86c9efc7 100644 --- a/sources/hv-collector-health-check/pom.xml +++ b/sources/hv-collector-health-check/pom.xml @@ -58,6 +58,12 @@ <artifactId>arrow-effects</artifactId> </dependency> <dependency> + <groupId>ch.qos.logback</groupId> + <artifactId>logback-classic</artifactId> + <scope>test</scope> + </dependency> + + <dependency> <groupId>org.jetbrains.kotlin</groupId> <artifactId>kotlin-test</artifactId> <scope>test</scope> diff --git a/sources/hv-collector-main/src/main/resources/logback.xml b/sources/hv-collector-main/src/main/resources/logback.xml index 58d2cb6a..c88b8aa8 100644 --- a/sources/hv-collector-main/src/main/resources/logback.xml +++ b/sources/hv-collector-main/src/main/resources/logback.xml @@ -1,4 +1,23 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> <property name="COMPONENT_NAME" value="dcae-hv-ves-collector"/> @@ -8,26 +27,53 @@ <property name="LOG_FILENAME" value="${COMPONENT_SHORT_NAME}"/> <property name="LOG_PATH" value="/var/log/ONAP/${COMPONENT_NAME}"/> <property name="ARCHIVE" value="${LOG_PATH}/archive"/> - <property name="FILE_LOG_PATTERN" value=" -%nopexception%50.50logger -| %date{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} -| %highlight(%-5level) -| %mdc -| %marker -| %msg -| %rootException -| %thread%n"/> + + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_thr" value="%thread"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="p_mdc" value="%replace(%replace(%mdc){'\t', '\\\\t'}){'\n', '\\\\n'}"/> + <property name="p_msg" value="%replace(%replace(%msg){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_exc" value="%replace(%replace(%rootException){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_mak" value="%replace(%replace(%marker){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| %rootException%n"/> + <property name="READABLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| ${p_mak}\t +| %rootException\t +| ${p_mdc}\t +| ${p_thr}%n"/> + <property name="ONAP_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_thr}\t +| ${p_lvl}\t +| ${p_log}\t +| ${p_mdc}\t +| ${p_msg}\t +| ${p_exc}\t +| ${p_mak}%n"/> <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> + <pattern>${READABLE_LOG_PATTERN}</pattern> </encoder> </appender> <appender name="ROLLING-FILE" class="ch.qos.logback.core.rolling.RollingFileAppender"> <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> + <pattern>${ONAP_LOG_PATTERN}</pattern> </encoder> <file>${LOG_PATH}/${LOG_FILENAME}.log</file> <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy"> diff --git a/sources/hv-collector-ssl/src/test/resources/logback-test.xml b/sources/hv-collector-ssl/src/test/resources/logback-test.xml index 9a4eacfe..400b1259 100644 --- a/sources/hv-collector-ssl/src/test/resources/logback-test.xml +++ b/sources/hv-collector-ssl/src/test/resources/logback-test.xml @@ -1,35 +1,43 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg%n"/> <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> + <pattern>${SIMPLE_LOG_PATTERN}</pattern> </encoder> </appender> - <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> - </appender> - <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> - </root> + <appender-ref ref="CONSOLE"/> + </root> </configuration> diff --git a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt index 1e98f2fc..95590d9d 100644 --- a/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt +++ b/sources/hv-collector-utils/src/main/kotlin/org/onap/dcae/collectors/veshv/utils/logging/reactive_logging.kt @@ -34,7 +34,6 @@ fun <T> Logger.handleReactiveStreamError( return returnFlux } - fun <T> Try<T>.filterFailedWithLog(logger: Logger, context: MappedDiagnosticContext, acceptedMsg: (T) -> String, diff --git a/sources/hv-collector-utils/src/test/resources/logback-test.xml b/sources/hv-collector-utils/src/test/resources/logback-test.xml index 9a4eacfe..400b1259 100644 --- a/sources/hv-collector-utils/src/test/resources/logback-test.xml +++ b/sources/hv-collector-utils/src/test/resources/logback-test.xml @@ -1,35 +1,43 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg%n"/> <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> + <pattern>${SIMPLE_LOG_PATTERN}</pattern> </encoder> </appender> - <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> - </appender> - <logger name="org.onap.dcae.collectors.veshv" level="TRACE"/> <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> - </root> + <appender-ref ref="CONSOLE"/> + </root> </configuration> diff --git a/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml b/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml index 809f62d4..2bc3f978 100644 --- a/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml +++ b/sources/hv-collector-xnf-simulator/src/main/resources/logback.xml @@ -1,35 +1,94 @@ <?xml version="1.0" encoding="UTF-8"?> +<!-- + ~ ============LICENSE_START======================================================= + ~ dcaegen2-collectors-veshv + ~ ================================================================================ + ~ Copyright (C) 2018 NOKIA + ~ ================================================================================ + ~ Licensed under the Apache License, Version 2.0 (the "License"); + ~ you may not use this file except in compliance with the License. + ~ You may obtain a copy of the License at + ~ + ~ http://www.apache.org/licenses/LICENSE-2.0 + ~ + ~ Unless required by applicable law or agreed to in writing, software + ~ distributed under the License is distributed on an "AS IS" BASIS, + ~ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + ~ See the License for the specific language governing permissions and + ~ limitations under the License. + ~ ============LICENSE_END========================================================= +--> <configuration> - <property name="LOG_FILE" - value="${LOG_FILE:-${LOG_PATH:-${LOG_TEMP:-${java.io.tmpdir:-/tmp}}/}ves-hv.log}"/> - <property name="FILE_LOG_PATTERN" value="%d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %-5level [%-40.40logger{10}] - %msg%n"/> + <property name="COMPONENT_NAME" + value="hv-ves-xnf-simulator"/> + <property name="COMPONENT_SHORT_NAME" + value="xnf-simulatr"/> - <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> - <encoder> - <pattern> - %d{yyyy-MM-dd'T'HH:mm:ss.SSSXXX,UTC} %highlight(%-5level) [%-40.40logger{10}] - %msg%n - </pattern> - </encoder> - </appender> + <property name="LOG_FILENAME" value="${COMPONENT_SHORT_NAME}"/> + <property name="LOG_PATH" value="/var/log/ONAP/${COMPONENT_NAME}"/> + <property name="ARCHIVE" value="${LOG_PATH}/archive"/> + + <property name="p_tim" value="%date{"yyyy-MM-dd'T'HH:mm:ss.SSSXXX", UTC}"/> + <property name="p_thr" value="%thread"/> + <property name="p_lvl" value="%highlight(%-5level)"/> + <property name="p_log" value="%50.50logger"/> + <property name="p_mdc" value="%replace(%replace(%mdc){'\t', '\\\\t'}){'\n', '\\\\n'}"/> + <property name="p_msg" value="%replace(%replace(%msg){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_exc" value="%replace(%replace(%rootException){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="p_mak" value="%replace(%replace(%marker){'\t', '\\\\t'}){'\n','\\\\n'}"/> + <property name="SIMPLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| %rootException%n"/> + <property name="READABLE_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_log}\t +| ${p_lvl}\t +| %msg\t +| ${p_mak}\t +| %rootException\t +| ${p_mdc}\t +| ${p_thr}%n"/> + <property name="ONAP_LOG_PATTERN" value=" +%nopexception +| ${p_tim}\t +| ${p_thr}\t +| ${p_lvl}\t +| ${p_log}\t +| ${p_mdc}\t +| ${p_msg}\t +| ${p_exc}\t +| ${p_mak}%n"/> + <property name="LOG_PATTERN_IN_USE" value="${SIMPLE_LOG_PATTERN}"/> + + <appender name="CONSOLE" class="ch.qos.logback.core.ConsoleAppender"> + <encoder> + <pattern>${LOG_PATTERN_IN_USE}</pattern> + </encoder> + </appender> <appender name="ROLLING-FILE" - class="ch.qos.logback.core.rolling.RollingFileAppender"> - <encoder> - <pattern>${FILE_LOG_PATTERN}</pattern> - </encoder> - <file>${LOG_FILE}</file> - <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> - <fileNamePattern>${LOG_FILE}.%d{yyyy-MM-dd}.log</fileNamePattern> - <maxFileSize>50MB</maxFileSize> - <maxHistory>30</maxHistory> - <totalSizeCap>10GB</totalSizeCap> - </rollingPolicy> + class="ch.qos.logback.core.rolling.RollingFileAppender"> + <encoder> + <pattern>${LOG_PATTERN_IN_USE}</pattern> + </encoder> + <file>${LOG_PATH}/${LOG_FILENAME}.log</file> + <rollingPolicy class="ch.qos.logback.core.rolling.SizeAndTimeBasedRollingPolicy"> + <FileNamePattern>${ARCHIVE}/${LOG_FILENAME}.%d{yyyy-MM-dd}.%i.log.gz</FileNamePattern> + <maxFileSize>50MB</maxFileSize> + <maxHistory>30</maxHistory> + <totalSizeCap>10GB</totalSizeCap> + </rollingPolicy> </appender> - <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> + <logger name="org.onap.dcae.collectors.veshv" level="DEBUG"/> - <root level="INFO"> - <appender-ref ref="CONSOLE"/> - <appender-ref ref="ROLLING-FILE"/> + <root level="INFO"> + <appender-ref ref="CONSOLE"/> + <appender-ref ref="ROLLING-FILE"/> </root> </configuration>
\ No newline at end of file |