From 941ec8b75c4321dcbe11c73fab9fa3ee769e834f Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Mon, 9 Aug 2021 21:14:04 +0400 Subject: [PATCH 01/41] [TH2-1950] Implemented new NoMessageCheck rule task --- README.md | 7 +- build.gradle | 4 +- gradle.properties | 2 +- .../exactpro/th2/check1/Check1Handler.java | 43 ++++-- .../com/exactpro/th2/check1/Checkpoint.java | 59 ++++---- .../exactpro/th2/check1/CheckpointData.java | 73 ++++++++++ .../th2/check1/CheckpointSubscriber.kt | 13 +- .../exactpro/th2/check1/CollectorService.kt | 69 ++++++--- .../th2/check1/rule/AbstractCheckTask.kt | 58 +++++--- .../th2/check1/rule/check/CheckRuleTask.kt | 3 +- .../rule/sequence/NoMessageCheckTask.kt | 134 ++++++++++++++++++ .../rule/sequence/SequenceCheckRuleTask.kt | 3 +- .../exactpro/th2/check1/utils/TimeUtils.kt | 35 +++++ .../exactpro/th2/check1/CheckpointTest.java | 30 ++-- .../com/exactpro/th2/check1/rule/TestChain.kt | 1 + .../check1/rule/check/TestCheckRuleTask.kt | 1 + 16 files changed, 439 insertions(+), 96 deletions(-) create mode 100644 src/main/java/com/exactpro/th2/check1/CheckpointData.java create mode 100644 src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt create mode 100644 src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt diff --git a/README.md b/README.md index 68cde7fc..04eeb851 100644 --- a/README.md +++ b/README.md @@ -1,4 +1,4 @@ -# th2 check1 (3.6.1) +# th2 check1 (3.7.0) ## Overview @@ -18,6 +18,7 @@ Available requests are described in [this repository](https://gitlab.exactpro.co - CheckSequenceRuleRequest - prefilters the messages and verify all of them by filter. Order checking configured from request. - CheckRuleRequest - get message filter from request and check it with messages in the cache or await specified time in case of empty cache or message absence. +- NoMessageCheckRequest - prefilters messages and verifies that no other messages have been received. ## Quick start General view of the component will look like this: @@ -110,6 +111,10 @@ spec: ## Release Notes +### 3.7.0 + ++ Implemented NoMessageCheck rule task. Updated CheckRule and CheckSequence rule tasks + ### 3.6.1 + Fixed a problem where rule completes check before the timer for execution has been scheduled diff --git a/build.gradle b/build.gradle index 09f09bc4..b282131b 100644 --- a/build.gradle +++ b/build.gradle @@ -165,8 +165,8 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.2.0' - implementation 'com.exactpro.th2:common:3.16.5' + implementation 'com.exactpro.th2:grpc-check1:3.3.0' + implementation 'com.exactpro.th2:common:3.21.1' implementation 'com.exactpro.th2:sailfish-utils:3.4.0' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" diff --git a/gradle.properties b/gradle.properties index 166fe0e0..c9b77d8b 100644 --- a/gradle.properties +++ b/gradle.properties @@ -1,4 +1,4 @@ -release_version = 3.6.1 +release_version = 3.7.0 description = 'th2 check1 box' diff --git a/src/main/java/com/exactpro/th2/check1/Check1Handler.java b/src/main/java/com/exactpro/th2/check1/Check1Handler.java index 3a517c3c..a5dc081a 100644 --- a/src/main/java/com/exactpro/th2/check1/Check1Handler.java +++ b/src/main/java/com/exactpro/th2/check1/Check1Handler.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2020 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) * 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 @@ -16,17 +16,11 @@ import static com.exactpro.th2.common.grpc.RequestStatus.Status.SUCCESS; import static com.google.protobuf.TextFormat.shortDebugString; +import com.exactpro.th2.check1.grpc.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.exactpro.th2.check1.grpc.ChainID; import com.exactpro.th2.check1.grpc.Check1Grpc.Check1ImplBase; -import com.exactpro.th2.check1.grpc.CheckRuleRequest; -import com.exactpro.th2.check1.grpc.CheckRuleResponse; -import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest; -import com.exactpro.th2.check1.grpc.CheckSequenceRuleResponse; -import com.exactpro.th2.check1.grpc.CheckpointRequest; -import com.exactpro.th2.check1.grpc.CheckpointResponse; import com.exactpro.th2.common.grpc.RequestStatus; import io.grpc.stub.StreamObserver; @@ -136,4 +130,37 @@ public void submitCheckSequenceRule(CheckSequenceRuleRequest request, StreamObse responseObserver.onCompleted(); } } + + @Override + public void submitNoMessageCheck(NoMessageCheckRequest request, StreamObserver responseObserver) { + try { + logger.info("Submitting sequence rule for request '{}' started", shortDebugString(request)); + + NoMessageCheckResponse.Builder response = NoMessageCheckResponse.newBuilder(); + try { + ChainID chainID = collectorService.verifyNoMessageCheck(request); + response.setChainId(chainID) + .setStatus(RequestStatus.newBuilder().setStatus(SUCCESS)); + } catch (Exception e) { + if (logger.isErrorEnabled()) { + logger.error("No message check rule task for request '" + shortDebugString(request) + "' isn't submitted", e); + } + RequestStatus status = RequestStatus.newBuilder() + .setStatus(ERROR) + .setMessage("No message check rule rejected by internal process: " + e.getMessage()) + .build(); + response.setStatus(status); + } + responseObserver.onNext(response.build()); + responseObserver.onCompleted(); + } catch (Throwable e) { + if (logger.isErrorEnabled()) { + logger.error("NoMessageCheck failed. Request " + shortDebugString(request), e); + } + responseObserver.onNext(NoMessageCheckResponse.newBuilder() + .setStatus(RequestStatus.newBuilder().setStatus(ERROR).setMessage("NoMessageCheck failed. See the logs.").build()) + .build()); + responseObserver.onCompleted(); + } + } } \ No newline at end of file diff --git a/src/main/java/com/exactpro/th2/check1/Checkpoint.java b/src/main/java/com/exactpro/th2/check1/Checkpoint.java index d94e5ef9..ce9a3edc 100644 --- a/src/main/java/com/exactpro/th2/check1/Checkpoint.java +++ b/src/main/java/com/exactpro/th2/check1/Checkpoint.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2020 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -17,28 +17,29 @@ import com.exactpro.th2.common.grpc.Checkpoint.DirectionCheckpoint; import com.exactpro.th2.common.grpc.Direction; +import com.google.protobuf.Timestamp; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import static com.datastax.driver.core.utils.UUIDs.timeBased; -import org.apache.commons.lang3.builder.EqualsBuilder; -import org.apache.commons.lang3.builder.HashCodeBuilder; -import org.apache.commons.lang3.builder.ToStringBuilder; +import static com.datastax.driver.core.utils.UUIDs.timeBased; public class Checkpoint { private final String id; - private final Map sessionKeyToSequence; + private final Map sessionKeyToCheckpointData; - private Checkpoint(String id, Map sessionKeyToSequence) { + private Checkpoint(String id, Map sessionKeyToCheckpointData) { this.id = id; - this.sessionKeyToSequence = Map.copyOf(sessionKeyToSequence); + this.sessionKeyToCheckpointData = Map.copyOf(sessionKeyToCheckpointData); } - public Checkpoint(Map sessionKeyToSequence) { - this(timeBased().toString(), sessionKeyToSequence); + public Checkpoint(Map sessionKeyToCheckpointData) { + this(timeBased().toString(), sessionKeyToCheckpointData); } public String getId() { @@ -46,20 +47,20 @@ public String getId() { } public boolean contains(SessionKey sessionKey) { - return sessionKeyToSequence.containsKey(sessionKey); + return sessionKeyToCheckpointData.containsKey(sessionKey); } - public long getSequence(SessionKey sessionKey) { - return sessionKeyToSequence.get(sessionKey); + public CheckpointData getCheckpointData(SessionKey sessionKey) { + return sessionKeyToCheckpointData.get(sessionKey); } public com.exactpro.th2.common.grpc.Checkpoint convert() { Map intermediateMap = new HashMap<>(); - for (Map.Entry entry : sessionKeyToSequence.entrySet()) { + for (Map.Entry entry : sessionKeyToCheckpointData.entrySet()) { SessionKey sessionKey = entry.getKey(); intermediateMap.computeIfAbsent(sessionKey.getSessionAlias(), alias -> DirectionCheckpoint.newBuilder()) - .putDirectionToSequence(sessionKey.getDirection().getNumber(), entry.getValue()); + .putDirectionToCheckpointData(sessionKey.getDirection().getNumber(), CheckpointData.convert(entry.getValue())); } var checkpointBuilder = com.exactpro.th2.common.grpc.Checkpoint.newBuilder() @@ -71,15 +72,15 @@ public com.exactpro.th2.common.grpc.Checkpoint convert() { return checkpointBuilder.build(); } - public Map asMap() { - return Collections.unmodifiableMap(sessionKeyToSequence); + public Map asMap() { + return Collections.unmodifiableMap(sessionKeyToCheckpointData); } @Override public String toString() { return new ToStringBuilder(this) .append("id", id) - .append("sessionKeyToSequence", sessionKeyToSequence) + .append("sessionKeyToSequence", sessionKeyToCheckpointData) .toString(); } @@ -97,7 +98,7 @@ public boolean equals(Object obj) { return new EqualsBuilder() .append(id, other.id) - .append(sessionKeyToSequence, other.sessionKeyToSequence) + .append(sessionKeyToCheckpointData, other.sessionKeyToCheckpointData) .isEquals(); } @@ -105,19 +106,27 @@ public boolean equals(Object obj) { public int hashCode() { return new HashCodeBuilder(17, 37) .append(id) - .append(sessionKeyToSequence) + .append(sessionKeyToCheckpointData) .toHashCode(); } public static Checkpoint convert(com.exactpro.th2.common.grpc.Checkpoint protoCheckpoint) { - Map sessionKeyToSequence = new HashMap<>(); + Map sessionKeyToSequence = new HashMap<>(); for (Map.Entry sessionAliasDirectionCheckpointEntry : protoCheckpoint.getSessionAliasToDirectionCheckpointMap().entrySet()) { String sessionAlias = sessionAliasDirectionCheckpointEntry.getKey(); DirectionCheckpoint directionCheckpoint = sessionAliasDirectionCheckpointEntry.getValue(); - for (Map.Entry directionSequenceEntry : directionCheckpoint.getDirectionToSequenceMap().entrySet()) { - SessionKey sessionKey = new SessionKey(sessionAlias, Direction.forNumber(directionSequenceEntry.getKey())); - sessionKeyToSequence.put(sessionKey, directionSequenceEntry.getValue()); - } + if (directionCheckpoint.getDirectionToCheckpointDataCount() != 0) { + for (Map.Entry directionSequenceEntry : directionCheckpoint.getDirectionToCheckpointDataMap().entrySet()) { + SessionKey sessionKey = new SessionKey(sessionAlias, Direction.forNumber(directionSequenceEntry.getKey())); + com.exactpro.th2.common.grpc.Checkpoint.CheckpointData checkpointData = directionSequenceEntry.getValue(); + sessionKeyToSequence.put(sessionKey, new CheckpointData(checkpointData.getSequence(), checkpointData.getTimestamp())); + } + } else { + for (Map.Entry directionSequenceEntry : directionCheckpoint.getDirectionToSequenceMap().entrySet()) { + SessionKey sessionKey = new SessionKey(sessionAlias, Direction.forNumber(directionSequenceEntry.getKey())); + sessionKeyToSequence.put(sessionKey, new CheckpointData(directionSequenceEntry.getValue(), Timestamp.getDefaultInstance())); + } + } } return new Checkpoint(protoCheckpoint.getId(), sessionKeyToSequence); } diff --git a/src/main/java/com/exactpro/th2/check1/CheckpointData.java b/src/main/java/com/exactpro/th2/check1/CheckpointData.java new file mode 100644 index 00000000..b6e1960a --- /dev/null +++ b/src/main/java/com/exactpro/th2/check1/CheckpointData.java @@ -0,0 +1,73 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1; + +import com.exactpro.th2.check1.utils.TimeUtilsKt; +import com.exactpro.th2.common.grpc.Checkpoint; +import com.google.protobuf.Timestamp; + +import java.time.Instant; +import java.util.Objects; + +public class CheckpointData { + private final Long sequence; + private final Timestamp timestamp; + + public CheckpointData(Long sequence, Timestamp timestamp) { + this.sequence = sequence; + this.timestamp = timestamp; + } + + public CheckpointData(Long sequence) { + this(sequence, null); + } + + public Long getSequence() { + return sequence; + } + + public Timestamp getTimestamp() { + return timestamp; + } + + public Instant getTimestampAsInstant() { + return TimeUtilsKt.toInstant(timestamp); + } + + public static CheckpointData convert(com.exactpro.th2.common.grpc.Checkpoint.CheckpointData checkpointData) { + return new CheckpointData(checkpointData.getSequence(), checkpointData.getTimestamp()); + } + + public static com.exactpro.th2.common.grpc.Checkpoint.CheckpointData convert(CheckpointData checkpointData) { + Checkpoint.CheckpointData.Builder builder = Checkpoint.CheckpointData.newBuilder() + .setSequence(checkpointData.getSequence()); + if (checkpointData.getTimestamp() != null) + builder.setTimestamp(checkpointData.getTimestamp()); + + return builder.build(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + CheckpointData that = (CheckpointData) o; + return Objects.equals(sequence, that.sequence) && Objects.equals(timestamp, that.timestamp); + } + + @Override + public int hashCode() { + return Objects.hash(sequence, timestamp); + } +} diff --git a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt index 35bb14ff..82126dfb 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2020 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) * 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 @@ -22,7 +22,14 @@ class CheckpointSubscriber : AbstractSessionObserver() { sessionSet += item } - fun createCheckpoint() : Checkpoint = Checkpoint(sessionSet.associateBy( + fun createCheckpoint(): Checkpoint = Checkpoint( + sessionSet.associateBy( { session -> session.sessionKey }, - { session -> session.lastMessage.metadata.id.sequence })) + { session -> + CheckpointData( + session.lastMessage.metadata.id.sequence, + session.lastMessage.metadata.timestamp + ) + }) + ) } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 922bd783..37c2a614 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -13,25 +13,14 @@ package com.exactpro.th2.check1 import com.exactpro.th2.check1.configuration.Check1Configuration -import com.exactpro.th2.check1.grpc.ChainID -import com.exactpro.th2.check1.grpc.CheckRuleRequest -import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest -import com.exactpro.th2.check1.grpc.CheckpointRequestOrBuilder +import com.exactpro.th2.check1.grpc.* import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.check.CheckRuleTask +import com.exactpro.th2.check1.rule.sequence.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.EventUtils -import com.exactpro.th2.common.grpc.ComparisonSettings -import com.exactpro.th2.common.grpc.ConnectionID -import com.exactpro.th2.common.grpc.Direction -import com.exactpro.th2.common.grpc.EventBatch -import com.exactpro.th2.common.grpc.EventID -import com.exactpro.th2.common.grpc.MessageBatch -import com.exactpro.th2.common.grpc.MessageFilter -import com.exactpro.th2.common.grpc.MessageID -import com.exactpro.th2.common.grpc.RootComparisonSettings -import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.grpc.* import com.exactpro.th2.common.schema.message.MessageListener import com.exactpro.th2.common.schema.message.MessageRouter import com.exactpro.th2.common.schema.message.SubscriberMonitor @@ -97,7 +86,8 @@ class CollectorService( val chainID = request.getChainIdOrGenerate() - val task = CheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), request.timeout, maxEventBatchContentSize, + val task = CheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), + request.messageTimeout, request.timeout, maxEventBatchContentSize, filter, parentEventID, streamObservable, eventBatchRouter) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -128,8 +118,39 @@ class CollectorService( } else { request.messageFiltersList.map { it.toRootMessageFilter() } } - val task = SequenceCheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), request.timeout, maxEventBatchContentSize, - request.preFilter, protoMessageFilters, request.checkOrder, parentEventID, streamObservable, eventBatchRouter) + + val task = SequenceCheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), + request.messageTimeout, request.timeout, maxEventBatchContentSize, request.preFilter, protoMessageFilters, + request.checkOrder, parentEventID, streamObservable, eventBatchRouter) + + cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) + + eventIdToLastCheckTask.compute(CheckTaskKey(chainID, request.connectivityId)) { _, value -> + task.apply { addToChainOrBegin(value, request.checkpoint) } + } + return chainID + } + + fun verifyNoMessageCheck(request: NoMessageCheckRequest): ChainID { + check(request.hasParentEventId()) { "Parent event id can't be null" } + val parentEventID: EventID = request.parentEventId + check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } + val sessionAlias: String = request.connectivityId.sessionAlias + val direction = directionOrDefault(request.direction) + val chainID = request.getChainIdOrGenerate() + + val task = NoMessageCheckTask( + request.description, + Instant.now(), + SessionKey(sessionAlias, direction), + request.messageTimeout, + request.timeout, + maxEventBatchContentSize, + request.preFilter, + parentEventID, + streamObservable, + eventBatchRouter + ) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -178,6 +199,14 @@ class CollectorService( } } + private fun NoMessageCheckRequest.getChainIdOrGenerate(): ChainID { + return if (hasChainId()) { + chainId + } else { + generateChainID() + } + } + private fun generateChainID() = ChainID.newBuilder().setId(EventUtils.generateUUID()).build() private fun cleanupTasksOlderThan(delta: Long, unit: ChronoUnit = ChronoUnit.SECONDS) { @@ -231,11 +260,11 @@ class CollectorService( val checkpoint = checkpointSubscriber.createCheckpoint() rootEvent.endTimestamp() .bodyData(EventUtils.createMessageBean("Checkpoint id '${checkpoint.id}'")) - checkpoint.asMap().forEach { (sessionKey: SessionKey, sequence: Long) -> - val messageID = sessionKey.toMessageID(sequence) + checkpoint.asMap().forEach { (sessionKey: SessionKey, checkpointData: CheckpointData) -> + val messageID = sessionKey.toMessageID(checkpointData.sequence) rootEvent.messageID(messageID) .addSubEventWithSamePeriod() - .name("Checkpoint for session alias '${sessionKey.sessionAlias}' direction '${sessionKey.direction}' sequence '$sequence'") + .name("Checkpoint for session alias '${sessionKey.sessionAlias}' direction '${sessionKey.direction}' sequence '${checkpointData.sequence}'") .type("Checkpoint for session") .messageID(messageID) } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 54f727ba..531197d9 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -18,22 +18,17 @@ import com.exactpro.sf.comparison.ComparisonResult import com.exactpro.sf.comparison.MessageComparator import com.exactpro.sf.scriptrunner.StatusType import com.exactpro.th2.check1.AbstractSessionObserver +import com.exactpro.th2.check1.CheckpointData import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.event.bean.builder.VerificationBuilder import com.exactpro.th2.check1.util.VerificationUtil +import com.exactpro.th2.check1.utils.toInstant import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.Event.Status.FAILED import com.exactpro.th2.common.event.Event.Status.PASSED import com.exactpro.th2.common.event.EventUtils -import com.exactpro.th2.common.grpc.Checkpoint -import com.exactpro.th2.common.grpc.EventBatch -import com.exactpro.th2.common.grpc.EventID -import com.exactpro.th2.common.grpc.Message -import com.exactpro.th2.common.grpc.MessageFilter -import com.exactpro.th2.common.grpc.MessageMetadata -import com.exactpro.th2.common.grpc.MetadataFilter -import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.grpc.* import com.exactpro.th2.common.message.toTreeTable import com.exactpro.th2.common.schema.message.MessageRouter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter @@ -58,6 +53,7 @@ import java.util.concurrent.atomic.AtomicReference */ abstract class AbstractCheckTask( val description: String?, + private val messageTimeout: Long? = null, private val timeout: Long, private val maxEventBatchContentSize: Int, submitTime: Instant, @@ -106,6 +102,7 @@ abstract class AbstractCheckTask( private lateinit var endFuture: Disposable private var lastSequence = DEFAULT_SEQUENCE + private var checkpointTimestamp: Instant? = null override fun onStart() { super.onStart() @@ -152,7 +149,7 @@ abstract class AbstractCheckTask( } else { legacy.executorService } - checkTask.begin(legacy.lastSequence, executor) + checkTask.begin(legacy.lastSequence, legacy.checkpointTimestamp, executor) } LOGGER.info("Task {} ({}) subscribed to task {} ({})", checkTask.description, checkTask.hashCode(), description, hashCode()) } else { @@ -164,11 +161,12 @@ abstract class AbstractCheckTask( * Observe a message sequence from the checkpoint. * Task subscribe to messages stream with its sequence after call. * This method should be called only once otherwise it throws IllegalStateException. - * @param checkpoint message sequence from previous task. + * @param checkpoint message sequence and checkpoint timestamp from previous task. * @throws IllegalStateException when method is called more than once. */ fun begin(checkpoint: Checkpoint? = null) { - begin(checkpoint?.getSequence(sessionKey) ?: DEFAULT_SEQUENCE) + val checkpointData = checkpoint?.getCheckpointData(sessionKey) + begin(checkpointData?.sequence ?: DEFAULT_SEQUENCE, checkpointData?.timestampAsInstant) } /** @@ -207,12 +205,15 @@ abstract class AbstractCheckTask( * @param executorService executor to schedule pipeline execution. * @throws IllegalStateException when method is called more than once. */ - private fun begin(sequence: Long = DEFAULT_SEQUENCE, executorService: ExecutorService = createExecutorService()) { + private fun begin(sequence: Long = DEFAULT_SEQUENCE, checkpointTimestamp: Instant? = null, + executorService: ExecutorService = createExecutorService()) { if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } LOGGER.info("Check begin for session alias '{}' with sequence '{}' timeout '{}'", sessionKey, sequence, timeout) this.lastSequence = sequence + if (checkpointTimestamp != null && messageTimeout != null) + this.checkpointTimestamp = checkpointTimestamp.plusMillis(messageTimeout) this.executorService = executorService val scheduler = Schedulers.from(executorService) @@ -235,8 +236,10 @@ abstract class AbstractCheckTask( .doOnNext { handledMessageCounter++ - with(it.metadata.id) { - rootEvent.messageID(this) + with(it.metadata) { + rootEvent.messageID(this.id) + if (checkOnMessageTimeout(this.timestamp.toInstant())) + checkComplete() } } .mapToMessageContainer() @@ -267,7 +270,7 @@ abstract class AbstractCheckTask( .toProto(parentEventID)) .build()) } finally { - sequenceSubject.onSuccess(Legacy(executorService, lastSequence)) + sequenceSubject.onSuccess(Legacy(executorService, lastSequence, checkpointTimestamp)) } } @@ -471,20 +474,29 @@ abstract class AbstractCheckTask( .flatMap(StreamContainer::bufferedMessages) .filter { message -> message.metadata.id.sequence > sequence } - private fun Checkpoint.getSequence(sessionKey: SessionKey): Long { - val sequence = sessionAliasToDirectionCheckpointMap[sessionKey.sessionAlias] - ?.directionToSequenceMap?.get(sessionKey.direction.number) + private fun Checkpoint.getCheckpointData(sessionKey: SessionKey): CheckpointData { + val checkpointData = sessionAliasToDirectionCheckpointMap[sessionKey.sessionAlias] + ?.directionToCheckpointDataMap?.get(sessionKey.direction.number) - if (sequence == null) { - if (LOGGER.isWarnEnabled) { + if (checkpointData == null) { + LOGGER.warn("Checkpoint '{}' doesn't contain checkpoint data for session '{}'", shortDebugString(this), sessionKey) + val sequence = sessionAliasToDirectionCheckpointMap[sessionKey.sessionAlias] + ?.directionToSequenceMap?.get(sessionKey.direction.number) + if (sequence == null) { LOGGER.warn("Checkpoint '{}' doesn't contain sequence for session '{}'", shortDebugString(this), sessionKey) + return CheckpointData(DEFAULT_SEQUENCE) } - } else { + return CheckpointData(sequence) + } + + return CheckpointData.convert(checkpointData).apply { LOGGER.info("Use sequence '{}' from checkpoint for session '{}'", sequence, sessionKey) } + } - return sequence ?: DEFAULT_SEQUENCE + private fun checkOnMessageTimeout(timestamp: Instant): Boolean { + return checkpointTimestamp != null && (checkpointTimestamp!!.isAfter(timestamp) || checkpointTimestamp == timestamp) } - private data class Legacy(val executorService: ExecutorService, val lastSequence: Long) + private data class Legacy(val executorService: ExecutorService, val lastSequence: Long, val checkpointTimestamp: Instant?) } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index 0bbed87c..c427762a 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -37,13 +37,14 @@ class CheckRuleTask( description: String?, startTime: Instant, sessionKey: SessionKey, + messageTimeout: Long? = null, timeout: Long, maxEventBatchContentSize: Int, private val protoMessageFilter: RootMessageFilter, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val messageFilter: SailfishFilter = SailfishFilter( converter.fromProtoFilter(protoMessageFilter.messageFilter, protoMessageFilter.messageType), diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt new file mode 100644 index 00000000..e395e1cf --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt @@ -0,0 +1,134 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.rule.sequence + +import com.exactpro.sf.common.messages.IMessage +import com.exactpro.th2.check1.SessionKey +import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.grpc.PreFilter +import com.exactpro.th2.check1.rule.AbstractCheckTask +import com.exactpro.th2.check1.rule.ComparisonContainer +import com.exactpro.th2.check1.rule.MessageContainer +import com.exactpro.th2.check1.rule.SailfishFilter +import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask.Companion.PRE_FILTER_MESSAGE_NAME +import com.exactpro.th2.check1.util.VerificationUtil +import com.exactpro.th2.common.event.Event +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.MessageFilter +import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.message.toTreeTable +import com.exactpro.th2.common.schema.message.MessageRouter +import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter +import com.google.protobuf.TextFormat +import io.reactivex.Observable +import java.time.Instant + +class NoMessageCheckTask( + description: String?, + startTime: Instant, + sessionKey: SessionKey, + messageTimeout: Long? = null, + timeout: Long, + maxEventBatchContentSize: Int, + protoPreFilter: PreFilter, + parentEventID: EventID, + messageStream: Observable, + eventBatchRouter: MessageRouter + ) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { + + private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() + private val messagePreFilter = SailfishFilter( + converter.fromProtoPreFilter(protoPreMessageFilter), + protoPreMessageFilter.toCompareSettings() + ) + + private val metadataPreFilter: SailfishFilter? = protoPreMessageFilter.metadataFilterOrNull()?.let { + SailfishFilter( + converter.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), + it.toComparisonSettings() + ) + } + + private lateinit var preFilterEvent: Event + private lateinit var resultEvent: Event + + private var preFilterMessagesCounter: Int = 0 + private var extraMessagesCounter: Int = 0 + + init { + rootEvent + .name("No message check $sessionKey") + .type("noMessageCheck") + } + + override fun onStart() { + super.onStart() + preFilterEvent = Event.start() + .type("preFiltering") + .bodyData(protoPreMessageFilter.toTreeTable()) + rootEvent.addSubEvent(preFilterEvent) + resultEvent = Event.start() + .type("noMessagesCheckResult") + rootEvent.addSubEvent(resultEvent) + } + + override fun onNext(messageContainer: MessageContainer) { + LOGGER.debug( + "Received message with id: {}", + TextFormat.shortDebugString(messageContainer.protoMessage.metadata.id) + ) + val result = matchFilter( + messageContainer, + messagePreFilter, + metadataPreFilter, + matchNames = false, + significant = false + ) + val comparisonContainer = ComparisonContainer(messageContainer, protoPreMessageFilter, result) + if (comparisonContainer.fullyMatches) { + preFilterMessagesCounter++ + preFilterEvent.messageID(comparisonContainer.protoActual.metadata.id) + } else { + extraMessagesCounter++ + resultEvent.messageID(comparisonContainer.protoActual.metadata.id) + } + } + + override fun completeEvent(canceled: Boolean) { + preFilterEvent.name("Prefilter: $preFilterMessagesCounter messages were filtered.") + + if (extraMessagesCounter == 0) + resultEvent.status(Event.Status.PASSED).name("Check passed") + else + resultEvent.status(Event.Status.FAILED).name("Check failed: $extraMessagesCounter extra messages were found.") + } + + private fun ProtoToIMessageConverter.fromProtoPreFilter(protoPreMessageFilter: RootMessageFilter): IMessage = + fromProtoFilter(protoPreMessageFilter.messageFilter, PRE_FILTER_MESSAGE_NAME) + + private fun PreFilter.toRootMessageFilter() = RootMessageFilter.newBuilder() + .setMessageType(PRE_FILTER_MESSAGE_NAME) + .setMessageFilter(toMessageFilter()) + .also { + if (hasMetadataFilter()) { + it.metadataFilter = metadataFilter + } + } + .build() + + private fun PreFilter.toMessageFilter() = MessageFilter.newBuilder() + .putAllFields(fieldsMap) + .build() +} \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index fad4a850..cf697361 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -61,6 +61,7 @@ class SequenceCheckRuleTask( description: String?, startTime: Instant, sessionKey: SessionKey, + messageTimeout: Long? = null, timeout: Long, maxEventBatchContentSize: Int, protoPreFilter: PreFilter, @@ -69,7 +70,7 @@ class SequenceCheckRuleTask( parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt new file mode 100644 index 00000000..1ddfdf9f --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt @@ -0,0 +1,35 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.utils + +import com.google.protobuf.Timestamp +import java.time.Instant + +fun Timestamp.toInstant(): Instant = Instant.ofEpochSecond(seconds, nanos.toLong()) +fun Instant.toTimestamp(): Timestamp = Timestamp.newBuilder().setSeconds(epochSecond).setNanos(nano).build() + +operator fun Timestamp.compareTo(other: Timestamp): Int { + if (this.seconds < other.seconds) + return -1 + if (this.seconds > other.seconds) + return 1 + if (this.nanos < other.nanos) + return -1 + if (this.nanos > other.nanos) + return 1 + return 0 +} + +fun Timestamp.isBefore(other: Timestamp): Boolean = this < other +fun Timestamp.isAfter(other: Timestamp): Boolean = this > other \ No newline at end of file diff --git a/src/test/java/com/exactpro/th2/check1/CheckpointTest.java b/src/test/java/com/exactpro/th2/check1/CheckpointTest.java index 038a129b..3671732e 100644 --- a/src/test/java/com/exactpro/th2/check1/CheckpointTest.java +++ b/src/test/java/com/exactpro/th2/check1/CheckpointTest.java @@ -12,22 +12,22 @@ */ package com.exactpro.th2.check1; -import java.util.Map; - +import com.exactpro.th2.common.grpc.Direction; +import com.google.protobuf.Timestamp; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import com.exactpro.th2.common.grpc.Direction; +import java.util.Map; public class CheckpointTest { @Test public void testConvertation() { var origCheckpoint = new Checkpoint(Map.of( - new SessionKey("A", Direction.FIRST), 1L, - new SessionKey("A", Direction.SECOND), 2L, - new SessionKey("B", Direction.FIRST), 3L, - new SessionKey("B", Direction.SECOND), 4L + new SessionKey("A", Direction.FIRST), generateCheckpointData(1L), + new SessionKey("A", Direction.SECOND), generateCheckpointData(2L), + new SessionKey("B", Direction.FIRST), generateCheckpointData(3L), + new SessionKey("B", Direction.SECOND), generateCheckpointData(4L) )); var protoCheckpoint = origCheckpoint.convert(); @@ -39,10 +39,18 @@ public void testConvertation() { private Checkpoint generateCheckpoint() { return new Checkpoint(Map.of( - new SessionKey("A", Direction.FIRST), 1L, - new SessionKey("A", Direction.FIRST), 2L, - new SessionKey("B", Direction.SECOND), 3L, - new SessionKey("B", Direction.SECOND), 4L + new SessionKey("A", Direction.FIRST), generateCheckpointData(1L), + new SessionKey("A", Direction.FIRST), generateCheckpointData(2L), + new SessionKey("B", Direction.SECOND), generateCheckpointData(3L), + new SessionKey("B", Direction.SECOND), generateCheckpointData(4L) )); } + + private CheckpointData generateCheckpointData(Long sequence, Timestamp timestamp) { + return new CheckpointData(sequence, timestamp); + } + + private CheckpointData generateCheckpointData(Long sequence) { + return generateCheckpointData(sequence, Timestamp.getDefaultInstance()); + } } \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index 1bfe47b3..47d96881 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -188,6 +188,7 @@ class TestChain: AbstractCheckTaskTest() { SESSION_ALIAS, Instant.now(), SessionKey(SESSION_ALIAS, FIRST), + null, 1000, maxEventBatchContentSize, createMessageFilter(sequence), diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index a3f15ce2..a6f5d10b 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -44,6 +44,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { SESSION_ALIAS, Instant.now(), SessionKey(SESSION_ALIAS, Direction.FIRST), + null, 1000, maxEventBatchContentSize, messageFilter, From 0dd47213a014db03f75e9ea1c8a82ffef93b10a1 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Tue, 10 Aug 2021 17:56:02 +0400 Subject: [PATCH 02/41] [TH2-1950] Replaced Instant with Timestamp * Fixed logging * Added proto message utils --- .../exactpro/th2/check1/Check1Handler.java | 26 ++++++++---- .../exactpro/th2/check1/CheckpointData.java | 12 ++---- .../th2/check1/rule/AbstractCheckTask.kt | 41 +++++++++++++------ .../rule/sequence/NoMessageCheckTask.kt | 41 ++++++------------- .../rule/sequence/SequenceCheckRuleTask.kt | 22 +--------- .../th2/check1/utils/ProtoMessageUtils.kt | 25 +++++++++++ .../exactpro/th2/check1/utils/TimeUtils.kt | 4 -- 7 files changed, 89 insertions(+), 82 deletions(-) create mode 100644 src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt diff --git a/src/main/java/com/exactpro/th2/check1/Check1Handler.java b/src/main/java/com/exactpro/th2/check1/Check1Handler.java index a5dc081a..4793fc0e 100644 --- a/src/main/java/com/exactpro/th2/check1/Check1Handler.java +++ b/src/main/java/com/exactpro/th2/check1/Check1Handler.java @@ -16,11 +16,20 @@ import static com.exactpro.th2.common.grpc.RequestStatus.Status.SUCCESS; import static com.google.protobuf.TextFormat.shortDebugString; -import com.exactpro.th2.check1.grpc.*; +import com.exactpro.th2.common.message.MessageUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.exactpro.th2.check1.grpc.ChainID; +import com.exactpro.th2.check1.grpc.CheckpointResponse; import com.exactpro.th2.check1.grpc.Check1Grpc.Check1ImplBase; +import com.exactpro.th2.check1.grpc.CheckRuleRequest; +import com.exactpro.th2.check1.grpc.CheckRuleResponse; +import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest; +import com.exactpro.th2.check1.grpc.CheckSequenceRuleResponse; +import com.exactpro.th2.check1.grpc.CheckpointRequest; +import com.exactpro.th2.check1.grpc.NoMessageCheckResponse; +import com.exactpro.th2.check1.grpc.NoMessageCheckRequest; import com.exactpro.th2.common.grpc.RequestStatus; import io.grpc.stub.StreamObserver; @@ -134,7 +143,9 @@ public void submitCheckSequenceRule(CheckSequenceRuleRequest request, StreamObse @Override public void submitNoMessageCheck(NoMessageCheckRequest request, StreamObserver responseObserver) { try { - logger.info("Submitting sequence rule for request '{}' started", shortDebugString(request)); + if (logger.isInfoEnabled()) { + logger.info("Submitting sequence rule for request '{}' started", MessageUtils.toJson(request)); + } NoMessageCheckResponse.Builder response = NoMessageCheckResponse.newBuilder(); try { @@ -143,19 +154,20 @@ public void submitNoMessageCheck(NoMessageCheckRequest request, StreamObserver Date: Wed, 11 Aug 2021 17:10:26 +0400 Subject: [PATCH 03/41] [TH2-1950] Small refactoring * Added new tests * Rewritten Checkpoint.java, CheckpointData.java classes in Kotlin * Added new extensions and utility classes * Fixed cancellation of a task by message timeout --- .../exactpro/th2/check1/Check1Handler.java | 10 +- .../com/exactpro/th2/check1/Checkpoint.java | 133 ------------ .../exactpro/th2/check1/CheckpointData.java | 67 ------ .../th2/check1/CheckpointSubscriber.kt | 9 +- .../exactpro/th2/check1/CollectorService.kt | 14 +- .../th2/check1/entities/Checkpoint.kt | 12 ++ .../th2/check1/entities/CheckpointData.kt | 5 + .../th2/check1/entities/TaskTimeout.kt | 5 + .../th2/check1/rule/AbstractCheckTask.kt | 22 +- .../th2/check1/rule/ComparisonContainer.kt | 20 +- .../th2/check1/rule/check/CheckRuleTask.kt | 6 +- .../rule/sequence/NoMessageCheckTask.kt | 24 ++- .../rule/sequence/SequenceCheckRuleTask.kt | 6 +- .../exactpro/th2/check1/utils/FilterUtils.kt | 37 ++++ .../th2/check1/utils/ProtoMessageUtils.kt | 67 +++++- .../exactpro/th2/check1/CheckpointTest.java | 7 +- .../th2/check1/rule/AbstractCheckTaskTest.kt | 38 +++- .../com/exactpro/th2/check1/rule/TestChain.kt | 6 +- .../check1/rule/check/TestCheckRuleTask.kt | 77 ++++++- .../rule/sequence/TestNoMessageCheckTask.kt | 191 ++++++++++++++++++ .../rule/sequence/TestSequenceCheckTask.kt | 162 ++++++++++++++- 21 files changed, 635 insertions(+), 283 deletions(-) delete mode 100644 src/main/java/com/exactpro/th2/check1/Checkpoint.java delete mode 100644 src/main/java/com/exactpro/th2/check1/CheckpointData.java create mode 100644 src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt create mode 100644 src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt create mode 100644 src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt create mode 100644 src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt create mode 100644 src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt diff --git a/src/main/java/com/exactpro/th2/check1/Check1Handler.java b/src/main/java/com/exactpro/th2/check1/Check1Handler.java index 4793fc0e..66fdf11b 100644 --- a/src/main/java/com/exactpro/th2/check1/Check1Handler.java +++ b/src/main/java/com/exactpro/th2/check1/Check1Handler.java @@ -16,6 +16,7 @@ import static com.exactpro.th2.common.grpc.RequestStatus.Status.SUCCESS; import static com.google.protobuf.TextFormat.shortDebugString; +import com.exactpro.th2.check1.utils.ProtoMessageUtilsKt; import com.exactpro.th2.common.message.MessageUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -48,7 +49,7 @@ public void createCheckpoint(CheckpointRequest request, StreamObserver sessionKeyToCheckpointData; - - private Checkpoint(String id, Map sessionKeyToCheckpointData) { - this.id = id; - this.sessionKeyToCheckpointData = Map.copyOf(sessionKeyToCheckpointData); - } - - public Checkpoint(Map sessionKeyToCheckpointData) { - this(timeBased().toString(), sessionKeyToCheckpointData); - } - - public String getId() { - return id; - } - - public boolean contains(SessionKey sessionKey) { - return sessionKeyToCheckpointData.containsKey(sessionKey); - } - - public CheckpointData getCheckpointData(SessionKey sessionKey) { - return sessionKeyToCheckpointData.get(sessionKey); - } - - public com.exactpro.th2.common.grpc.Checkpoint convert() { - Map intermediateMap = new HashMap<>(); - - for (Map.Entry entry : sessionKeyToCheckpointData.entrySet()) { - SessionKey sessionKey = entry.getKey(); - intermediateMap.computeIfAbsent(sessionKey.getSessionAlias(), alias -> DirectionCheckpoint.newBuilder()) - .putDirectionToCheckpointData(sessionKey.getDirection().getNumber(), CheckpointData.convert(entry.getValue())); - } - - var checkpointBuilder = com.exactpro.th2.common.grpc.Checkpoint.newBuilder() - .setId(id); - for (Map.Entry entry : intermediateMap.entrySet()) { - checkpointBuilder.putSessionAliasToDirectionCheckpoint(entry.getKey(), - entry.getValue().build()); - } - return checkpointBuilder.build(); - } - - public Map asMap() { - return Collections.unmodifiableMap(sessionKeyToCheckpointData); - } - - @Override - public String toString() { - return new ToStringBuilder(this) - .append("id", id) - .append("sessionKeyToSequence", sessionKeyToCheckpointData) - .toString(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - Checkpoint other = (Checkpoint)obj; - - return new EqualsBuilder() - .append(id, other.id) - .append(sessionKeyToCheckpointData, other.sessionKeyToCheckpointData) - .isEquals(); - } - - @Override - public int hashCode() { - return new HashCodeBuilder(17, 37) - .append(id) - .append(sessionKeyToCheckpointData) - .toHashCode(); - } - - public static Checkpoint convert(com.exactpro.th2.common.grpc.Checkpoint protoCheckpoint) { - Map sessionKeyToSequence = new HashMap<>(); - for (Map.Entry sessionAliasDirectionCheckpointEntry : protoCheckpoint.getSessionAliasToDirectionCheckpointMap().entrySet()) { - String sessionAlias = sessionAliasDirectionCheckpointEntry.getKey(); - DirectionCheckpoint directionCheckpoint = sessionAliasDirectionCheckpointEntry.getValue(); - if (directionCheckpoint.getDirectionToCheckpointDataCount() != 0) { - for (Map.Entry directionSequenceEntry : directionCheckpoint.getDirectionToCheckpointDataMap().entrySet()) { - SessionKey sessionKey = new SessionKey(sessionAlias, Direction.forNumber(directionSequenceEntry.getKey())); - com.exactpro.th2.common.grpc.Checkpoint.CheckpointData checkpointData = directionSequenceEntry.getValue(); - sessionKeyToSequence.put(sessionKey, new CheckpointData(checkpointData.getSequence(), checkpointData.getTimestamp())); - } - } else { - for (Map.Entry directionSequenceEntry : directionCheckpoint.getDirectionToSequenceMap().entrySet()) { - SessionKey sessionKey = new SessionKey(sessionAlias, Direction.forNumber(directionSequenceEntry.getKey())); - sessionKeyToSequence.put(sessionKey, new CheckpointData(directionSequenceEntry.getValue(), Timestamp.getDefaultInstance())); - } - } - } - return new Checkpoint(protoCheckpoint.getId(), sessionKeyToSequence); - } -} diff --git a/src/main/java/com/exactpro/th2/check1/CheckpointData.java b/src/main/java/com/exactpro/th2/check1/CheckpointData.java deleted file mode 100644 index 85c4650c..00000000 --- a/src/main/java/com/exactpro/th2/check1/CheckpointData.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) - * 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. - */ - -package com.exactpro.th2.check1; - -import com.exactpro.th2.common.grpc.Checkpoint; -import com.google.protobuf.Timestamp; - -import java.util.Objects; - -public class CheckpointData { - private final long sequence; - private final Timestamp timestamp; - - public CheckpointData(Long sequence, Timestamp timestamp) { - this.sequence = sequence; - this.timestamp = timestamp; - } - - public CheckpointData(long sequence) { - this(sequence, null); - } - - public long getSequence() { - return sequence; - } - - public Timestamp getTimestamp() { - return timestamp; - } - - public static CheckpointData convert(com.exactpro.th2.common.grpc.Checkpoint.CheckpointData checkpointData) { - return new CheckpointData(checkpointData.getSequence(), checkpointData.getTimestamp()); - } - - public static com.exactpro.th2.common.grpc.Checkpoint.CheckpointData convert(CheckpointData checkpointData) { - Checkpoint.CheckpointData.Builder builder = Checkpoint.CheckpointData.newBuilder() - .setSequence(checkpointData.getSequence()); - if (checkpointData.getTimestamp() != null) - builder.setTimestamp(checkpointData.getTimestamp()); - - return builder.build(); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - CheckpointData that = (CheckpointData) o; - return Objects.equals(sequence, that.sequence) && Objects.equals(timestamp, that.timestamp); - } - - @Override - public int hashCode() { - return Objects.hash(sequence, timestamp); - } -} diff --git a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt index 82126dfb..786b5f86 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt @@ -13,6 +13,8 @@ package com.exactpro.th2.check1 +import com.exactpro.th2.check1.entities.Checkpoint +import com.exactpro.th2.check1.entities.CheckpointData import java.util.concurrent.ConcurrentHashMap class CheckpointSubscriber : AbstractSessionObserver() { @@ -26,10 +28,9 @@ class CheckpointSubscriber : AbstractSessionObserver() { sessionSet.associateBy( { session -> session.sessionKey }, { session -> - CheckpointData( - session.lastMessage.metadata.id.sequence, - session.lastMessage.metadata.timestamp - ) + session.lastMessage.metadata.run { + CheckpointData(id.sequence, timestamp) + } }) ) } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 37c2a614..f85ab63a 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -13,6 +13,9 @@ package com.exactpro.th2.check1 import com.exactpro.th2.check1.configuration.Check1Configuration +import com.exactpro.th2.check1.entities.Checkpoint +import com.exactpro.th2.check1.entities.CheckpointData +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.* import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.check.CheckRuleTask @@ -87,7 +90,7 @@ class CollectorService( val chainID = request.getChainIdOrGenerate() val task = CheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), - request.messageTimeout, request.timeout, maxEventBatchContentSize, + TaskTimeout(request.messageTimeout, request.timeout), maxEventBatchContentSize, filter, parentEventID, streamObservable, eventBatchRouter) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -120,8 +123,8 @@ class CollectorService( } val task = SequenceCheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), - request.messageTimeout, request.timeout, maxEventBatchContentSize, request.preFilter, protoMessageFilters, - request.checkOrder, parentEventID, streamObservable, eventBatchRouter) + TaskTimeout(request.messageTimeout, request.timeout), maxEventBatchContentSize, request.preFilter, + protoMessageFilters, request.checkOrder, parentEventID, streamObservable, eventBatchRouter) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -143,8 +146,7 @@ class CollectorService( request.description, Instant.now(), SessionKey(sessionAlias, direction), - request.messageTimeout, - request.timeout, + TaskTimeout(request.messageTimeout, request.timeout), maxEventBatchContentSize, request.preFilter, parentEventID, @@ -260,7 +262,7 @@ class CollectorService( val checkpoint = checkpointSubscriber.createCheckpoint() rootEvent.endTimestamp() .bodyData(EventUtils.createMessageBean("Checkpoint id '${checkpoint.id}'")) - checkpoint.asMap().forEach { (sessionKey: SessionKey, checkpointData: CheckpointData) -> + checkpoint.sessionKeyToCheckpointData.forEach { (sessionKey: SessionKey, checkpointData: CheckpointData) -> val messageID = sessionKey.toMessageID(checkpointData.sequence) rootEvent.messageID(messageID) .addSubEventWithSamePeriod() diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt new file mode 100644 index 00000000..fe538260 --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt @@ -0,0 +1,12 @@ +package com.exactpro.th2.check1.entities + +import com.datastax.driver.core.utils.UUIDs.timeBased +import com.exactpro.th2.check1.SessionKey + +data class Checkpoint( + val id: String = timeBased().toString(), val sessionKeyToCheckpointData: Map +) { + constructor(sessionKeyToCheckpointData: Map) : this( + timeBased().toString(), sessionKeyToCheckpointData + ) +} \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt new file mode 100644 index 00000000..5225af3e --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt @@ -0,0 +1,5 @@ +package com.exactpro.th2.check1.entities + +import com.google.protobuf.Timestamp + +data class CheckpointData(val sequence: Long, val timestamp: Timestamp? = null) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt new file mode 100644 index 00000000..8be40f32 --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt @@ -0,0 +1,5 @@ +package com.exactpro.th2.check1.entities + +data class TaskTimeout(val messageTimeout: Long? = null, val timeout: Long) { + constructor(timeout: Long) : this(null, timeout) +} \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 257c9dea..d2a6b590 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -18,11 +18,14 @@ import com.exactpro.sf.comparison.ComparisonResult import com.exactpro.sf.comparison.MessageComparator import com.exactpro.sf.scriptrunner.StatusType import com.exactpro.th2.check1.AbstractSessionObserver -import com.exactpro.th2.check1.CheckpointData import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.CheckpointData +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.event.bean.builder.VerificationBuilder import com.exactpro.th2.check1.util.VerificationUtil +import com.exactpro.th2.check1.utils.convert +import com.exactpro.th2.check1.utils.getStatusType import com.exactpro.th2.check1.utils.isAfter import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.Event.Status.FAILED @@ -57,8 +60,7 @@ import java.util.concurrent.atomic.AtomicReference */ abstract class AbstractCheckTask( val description: String?, - private val messageTimeout: Long? = null, - private val timeout: Long, + private val taskTimeout: TaskTimeout, private val maxEventBatchContentSize: Int, submitTime: Instant, protected val sessionKey: SessionKey, @@ -214,13 +216,13 @@ abstract class AbstractCheckTask( if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } - LOGGER.info("Check begin for session alias '{}' with sequence '{}' timeout '{}'", sessionKey, sequence, timeout) + LOGGER.info("Check begin for session alias '{}' with sequence '{}' timeout '{}'", sessionKey, sequence, taskTimeout.timeout) this.lastSequence = sequence - this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, messageTimeout) + this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) this.executorService = executorService val scheduler = Schedulers.from(executorService) - endFuture = Single.timer(timeout, MILLISECONDS, Schedulers.computation()) + endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) .subscribe { _ -> end("Timeout is exited") } messageStream.observeOn(scheduler) // Defined scheduler to execution in one thread to avoid race-condition. @@ -241,11 +243,9 @@ abstract class AbstractCheckTask( with(it.metadata) { rootEvent.messageID(this.id) - if (checkOnMessageTimeout(this.timestamp)) { - checkComplete() - } } } + .takeWhile { checkOnMessageTimeout(it.metadata.timestamp) } .mapToMessageContainer() .taskPipeline() .subscribe(this) @@ -497,13 +497,13 @@ abstract class AbstractCheckTask( return CheckpointData(sequence) } - return CheckpointData.convert(checkpointData).apply { + return checkpointData.convert().apply { LOGGER.info("Use sequence '{}' from checkpoint for session '{}'", sequence, sessionKey) } } private fun checkOnMessageTimeout(timestamp: Timestamp): Boolean { - return checkpointTimeout != null && (checkpointTimeout!!.isAfter(timestamp) || checkpointTimeout == timestamp) + return checkpointTimeout == null || checkpointTimeout!!.isAfter(timestamp) || checkpointTimeout == timestamp } private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? = diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/ComparisonContainer.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/ComparisonContainer.kt index 9ac69387..0f5b2101 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/ComparisonContainer.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/ComparisonContainer.kt @@ -14,8 +14,9 @@ package com.exactpro.th2.check1.rule import com.exactpro.sf.common.messages.IMessage import com.exactpro.sf.comparison.ComparisonResult -import com.exactpro.sf.comparison.ComparisonUtil import com.exactpro.sf.scriptrunner.StatusType +import com.exactpro.th2.check1.utils.FilterUtils +import com.exactpro.th2.check1.utils.FilterUtils.fullMatch import com.exactpro.th2.common.grpc.Message import com.exactpro.th2.common.grpc.RootMessageFilter @@ -37,7 +38,7 @@ class ComparisonContainer( * Otherwise, returns `true` only if [AggregatedFilterResult.messageResult] is not `null` */ val matchesByKeys: Boolean - get() = result.allMatches { it != null } + get() = FilterUtils.allMatches(result, protoFilter) { it != null } /** * If [RootMessageFilter.hasMetadataFilter] for [protoFilter] is `true` @@ -47,16 +48,7 @@ class ComparisonContainer( * and its aggregated status is not [StatusType.FAILED] */ val fullyMatches: Boolean - get() = result.allMatches { it.fullMatch } - - private fun AggregatedFilterResult.allMatches(test: (ComparisonResult?) -> Boolean): Boolean { - return test(messageResult) && (!protoFilter.hasMetadataFilter() || test(metadataResult)) - } - - companion object { - private val ComparisonResult?.fullMatch: Boolean - get() = this != null && getStatusType() != StatusType.FAILED - } + get() = FilterUtils.allMatches(result, protoFilter) { it.fullMatch } } class AggregatedFilterResult( @@ -69,6 +61,4 @@ class AggregatedFilterResult( @JvmField val EMPTY = AggregatedFilterResult(null, null) } -} - -fun ComparisonResult.getStatusType(): StatusType = ComparisonUtil.getStatusType(this) \ No newline at end of file +} \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index c427762a..36cd2050 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -15,6 +15,7 @@ package com.exactpro.th2.check1.rule.check import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer @@ -37,14 +38,13 @@ class CheckRuleTask( description: String?, startTime: Instant, sessionKey: SessionKey, - messageTimeout: Long? = null, - timeout: Long, + taskTimeout: TaskTimeout, maxEventBatchContentSize: Int, private val protoMessageFilter: RootMessageFilter, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val messageFilter: SailfishFilter = SailfishFilter( converter.fromProtoFilter(protoMessageFilter.messageFilter, protoMessageFilter.messageType), diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt index a9e39f9a..ace89c52 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt @@ -15,12 +15,14 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask -import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter import com.exactpro.th2.check1.util.VerificationUtil +import com.exactpro.th2.check1.utils.FilterUtils +import com.exactpro.th2.check1.utils.FilterUtils.fullMatch import com.exactpro.th2.check1.utils.fromProtoPreFilter import com.exactpro.th2.check1.utils.toRootMessageFilter import com.exactpro.th2.common.event.Event @@ -37,14 +39,13 @@ class NoMessageCheckTask( description: String?, startTime: Instant, sessionKey: SessionKey, - messageTimeout: Long? = null, - timeout: Long, + taskTimeout: TaskTimeout, maxEventBatchContentSize: Int, protoPreFilter: PreFilter, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter - ) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { + ) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( @@ -94,13 +95,14 @@ class NoMessageCheckTask( matchNames = false, significant = false ) - val comparisonContainer = ComparisonContainer(messageContainer, protoPreMessageFilter, result) - if (comparisonContainer.fullyMatches) { - preFilterMessagesCounter++ - preFilterEvent.messageID(comparisonContainer.protoActual.metadata.id) - } else { - extraMessagesCounter++ - resultEvent.messageID(comparisonContainer.protoActual.metadata.id) + messageContainer.protoMessage.metadata.run { + if (FilterUtils.allMatches(result, protoPreMessageFilter) { it.fullMatch }) { + preFilterMessagesCounter++ + preFilterEvent.messageID(id) + } else { + extraMessagesCounter++ + resultEvent.messageID(id) + } } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 8923dc97..407bf9c6 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -17,6 +17,7 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.event.CheckSequenceUtils import com.exactpro.th2.check1.event.bean.CheckSequenceRow import com.exactpro.th2.check1.grpc.PreFilter @@ -60,8 +61,7 @@ class SequenceCheckRuleTask( description: String?, startTime: Instant, sessionKey: SessionKey, - messageTimeout: Long? = null, - timeout: Long, + taskTimeout: TaskTimeout, maxEventBatchContentSize: Int, protoPreFilter: PreFilter, private val protoMessageFilters: List, @@ -69,7 +69,7 @@ class SequenceCheckRuleTask( parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, messageTimeout, timeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt new file mode 100644 index 00000000..0638b864 --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt @@ -0,0 +1,37 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.utils + +import com.exactpro.sf.comparison.ComparisonResult +import com.exactpro.sf.comparison.ComparisonUtil +import com.exactpro.sf.scriptrunner.StatusType +import com.exactpro.th2.check1.rule.AggregatedFilterResult +import com.exactpro.th2.common.grpc.RootMessageFilter + +object FilterUtils { + @JvmStatic + val ComparisonResult?.fullMatch: Boolean + get() = this != null && getStatusType() != StatusType.FAILED + + @JvmStatic + fun allMatches( + aggregatedFilterResult: AggregatedFilterResult, + protoFilter: RootMessageFilter, + condition: (ComparisonResult?) -> Boolean + ): Boolean = condition(aggregatedFilterResult.messageResult) && (!protoFilter.hasMetadataFilter() || condition( + aggregatedFilterResult.metadataResult + )) +} + +fun ComparisonResult.getStatusType(): StatusType = ComparisonUtil.getStatusType(this) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt index 42069fca..8c752a31 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt @@ -1,8 +1,26 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + package com.exactpro.th2.check1.utils import com.exactpro.sf.common.messages.IMessage +import com.exactpro.th2.check1.SessionKey +import com.exactpro.th2.check1.entities.CheckpointData import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask +import com.exactpro.th2.common.grpc.Checkpoint +import com.exactpro.th2.common.grpc.Checkpoint.DirectionCheckpoint +import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter @@ -22,4 +40,51 @@ fun PreFilter.toRootMessageFilter(): RootMessageFilter = RootMessageFilter.newBu fun PreFilter.toMessageFilter(): MessageFilter = MessageFilter.newBuilder() .putAllFields(fieldsMap) - .build() \ No newline at end of file + .build() + +fun Checkpoint.CheckpointData.convert(): CheckpointData = CheckpointData(sequence, timestamp) + +fun CheckpointData.convert(): Checkpoint.CheckpointData { + val builder = Checkpoint.CheckpointData.newBuilder().setSequence(sequence) + if (timestamp != null) + builder.timestamp = timestamp + return builder.build() +} + +fun com.exactpro.th2.check1.entities.Checkpoint.convert(): Checkpoint { + val intermediateMap: MutableMap = HashMap() + sessionKeyToCheckpointData.forEach { (sessionKey, checkpointData) -> + intermediateMap.computeIfAbsent(sessionKey.sessionAlias) { + DirectionCheckpoint.newBuilder() + }.putDirectionToCheckpointData(sessionKey.direction.number, checkpointData.convert()) + } + + val checkpointBuilder = Checkpoint.newBuilder().setId(id) + intermediateMap.forEach { (sessionAlias, directionCheckpoint) -> + checkpointBuilder.putSessionAliasToDirectionCheckpoint(sessionAlias, directionCheckpoint.build()) + } + + return checkpointBuilder.build() +} + +fun Checkpoint.convert(): com.exactpro.th2.check1.entities.Checkpoint { + val sessionKeyToSequence: MutableMap = HashMap() + sessionAliasToDirectionCheckpointMap.forEach { (sessionAlias, directionCheckpoint) -> + check(!(directionCheckpoint.directionToCheckpointDataCount != 0 && directionCheckpoint.directionToSequenceCount != 0)) { + "Session alias '${sessionAlias}' cannot contain both of these fields: 'direction to checkpoint data' and 'direction to sequence'. Please use 'direction to checkpoint data' instead" + } + if (directionCheckpoint.directionToCheckpointDataCount == 0) { + directionCheckpoint.directionToSequenceMap.forEach { (directionNumber, sequence) -> + val sessionKey = SessionKey(sessionAlias, Direction.forNumber(directionNumber)) + sessionKeyToSequence[sessionKey] = CheckpointData(sequence, null) + } + } else { + directionCheckpoint.directionToCheckpointDataMap.forEach { (directionNumber, checkpointData) -> + val sessionKey = SessionKey(sessionAlias, Direction.forNumber(directionNumber)) + sessionKeyToSequence[sessionKey] = checkpointData.convert() + + } + } + } + return com.exactpro.th2.check1.entities.Checkpoint(id, sessionKeyToSequence) +} \ No newline at end of file diff --git a/src/test/java/com/exactpro/th2/check1/CheckpointTest.java b/src/test/java/com/exactpro/th2/check1/CheckpointTest.java index 3671732e..8b1d84de 100644 --- a/src/test/java/com/exactpro/th2/check1/CheckpointTest.java +++ b/src/test/java/com/exactpro/th2/check1/CheckpointTest.java @@ -12,6 +12,9 @@ */ package com.exactpro.th2.check1; +import com.exactpro.th2.check1.entities.Checkpoint; +import com.exactpro.th2.check1.entities.CheckpointData; +import com.exactpro.th2.check1.utils.ProtoMessageUtilsKt; import com.exactpro.th2.common.grpc.Direction; import com.google.protobuf.Timestamp; import org.junit.jupiter.api.Assertions; @@ -30,9 +33,9 @@ public void testConvertation() { new SessionKey("B", Direction.SECOND), generateCheckpointData(4L) )); - var protoCheckpoint = origCheckpoint.convert(); + var protoCheckpoint = ProtoMessageUtilsKt.convert(origCheckpoint); - var parsedCheckpoint = Checkpoint.convert(protoCheckpoint); + var parsedCheckpoint = ProtoMessageUtilsKt.convert(protoCheckpoint); Assertions.assertEquals(origCheckpoint, parsedCheckpoint); } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index e46cf455..168b9ce6 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -17,13 +17,17 @@ import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.Direction.FIRST import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.Message +import com.exactpro.th2.common.message.toTimestamp import com.exactpro.th2.common.schema.message.MessageRouter +import com.google.protobuf.Timestamp import com.nhaarman.mockitokotlin2.argumentCaptor import com.nhaarman.mockitokotlin2.spy import com.nhaarman.mockitokotlin2.timeout import com.nhaarman.mockitokotlin2.verify import io.reactivex.Observable +import java.time.Instant abstract class AbstractCheckTaskTest { protected val clientStub: MessageRouter = spy { } @@ -40,9 +44,16 @@ abstract class AbstractCheckTaskTest { ) } - fun constructMessage(sequence: Long = 0, alias: String = SESSION_ALIAS, type: String = MESSAGE_TYPE, direction: Direction = FIRST): Message.Builder = Message.newBuilder().apply { + fun constructMessage( + sequence: Long = 0, + alias: String = SESSION_ALIAS, + type: String = MESSAGE_TYPE, + direction: Direction = FIRST, + timestamp: Timestamp? = Timestamp.getDefaultInstance() + ): Message.Builder = Message.newBuilder().apply { metadataBuilder.apply { - messageType = type + this.messageType = type + this.timestamp = timestamp ?: Timestamp.getDefaultInstance() idBuilder.apply { this.sequence = sequence this.direction = direction @@ -51,6 +62,29 @@ abstract class AbstractCheckTaskTest { } } + protected fun createEvent(id: String): EventID { + return EventID.newBuilder().setId(id).build() + } + + protected fun getMessageTimestamp(start: Instant, delta: Long): Timestamp = + start.plusMillis(delta).toTimestamp() + + protected fun createCheckpoint(timestamp: Instant, sequence: Long = -1) : com.exactpro.th2.common.grpc.Checkpoint = + com.exactpro.th2.common.grpc.Checkpoint.newBuilder().apply { + putSessionAliasToDirectionCheckpoint( + SESSION_ALIAS, + com.exactpro.th2.common.grpc.Checkpoint.DirectionCheckpoint.newBuilder().apply { + putDirectionToCheckpointData( + FIRST.number, + com.exactpro.th2.common.grpc.Checkpoint.CheckpointData.newBuilder().apply { + this.sequence = sequence + this.timestamp = timestamp.toTimestamp() + }.build() + ) + }.build() + ) + }.build() + companion object { const val MESSAGE_TYPE = "TestMsg" const val SESSION_ALIAS = "test_session" diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index 47d96881..144ba21d 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -14,6 +14,7 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.check.CheckRuleTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask @@ -168,7 +169,7 @@ class TestChain: AbstractCheckTaskTest() { description = "Test", startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, FIRST), - timeout = 1000L, + taskTimeout = TaskTimeout(null, 1000L), maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = sequence.map(::createMessageFilter).toList(), @@ -188,8 +189,7 @@ class TestChain: AbstractCheckTaskTest() { SESSION_ALIAS, Instant.now(), SessionKey(SESSION_ALIAS, FIRST), - null, - 1000, + TaskTimeout(1000L), maxEventBatchContentSize, createMessageFilter(sequence), parentEventID, diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index a6f5d10b..1becf916 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -15,14 +15,17 @@ package com.exactpro.th2.check1.rule.check import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.rule.AbstractCheckTaskTest import com.exactpro.th2.check1.util.toSimpleFilter import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.EventStatus +import com.exactpro.th2.common.grpc.EventStatus.FAILED import com.exactpro.th2.common.grpc.EventStatus.SUCCESS import com.exactpro.th2.common.grpc.FilterOperation +import com.exactpro.th2.common.grpc.MessageID import com.exactpro.th2.common.grpc.MessageMetadata import com.exactpro.th2.common.grpc.MetadataFilter import com.exactpro.th2.common.grpc.RootMessageFilter @@ -39,13 +42,13 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { messageFilter: RootMessageFilter, parentEventID: EventID, messageStream: Observable, - maxEventBatchContentSize: Int = 1024 * 1024 + maxEventBatchContentSize: Int = 1024 * 1024, + taskTimeout: TaskTimeout = TaskTimeout(1000L) ) = CheckRuleTask( SESSION_ALIAS, Instant.now(), SessionKey(SESSION_ALIAS, Direction.FIRST), - null, - 1000, + taskTimeout, maxEventBatchContentSize, messageFilter, parentEventID, @@ -64,7 +67,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .build() )) - val eventID = EventID.newBuilder().setId("root").build() + val eventID = createEvent("root") val filter = RootMessageFilter.newBuilder() .setMessageType(MESSAGE_TYPE) .setMetadataFilter(MetadataFilter.newBuilder() @@ -90,7 +93,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .build() )) - val eventID = EventID.newBuilder().setId("root").build() + val eventID = createEvent("root") val filter = RootMessageFilter.newBuilder() .setMessageType(MESSAGE_TYPE) .setMetadataFilter(MetadataFilter.newBuilder() @@ -116,7 +119,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .build() )) - val eventID = EventID.newBuilder().setId("root").build() + val eventID = createEvent("root") val filter = RootMessageFilter.newBuilder() .setMessageType(MESSAGE_TYPE) .setMetadataFilter(MetadataFilter.newBuilder() @@ -142,7 +145,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .build() )) - val eventID = EventID.newBuilder().setId("root").build() + val eventID = createEvent("root") val filter = RootMessageFilter.newBuilder() .setMessageType(MESSAGE_TYPE) .setMetadataFilter(MetadataFilter.newBuilder() @@ -172,7 +175,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .build() )) - val eventID = EventID.newBuilder().setId("root").build() + val eventID = createEvent("root") val filter = RootMessageFilter.newBuilder() .setMessageType(MESSAGE_TYPE) .setMetadataFilter(MetadataFilter.newBuilder() @@ -190,4 +193,62 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { "No failed event $eventBatch" } } + + @Test + fun `success verification with message timeout`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(SESSION_ALIAS, Direction.FIRST, listOf( + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setTimestamp(getMessageTimestamp(checkpointTimestamp, 500)) + .setId(MessageID.newBuilder().setSequence(1L)) + .build()) + .build() + )) + + val eventID = createEvent("root") + val filter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMetadataFilter(MetadataFilter.newBuilder() + .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) + .build() + val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(500, 1000)) + task.begin(createCheckpoint(checkpointTimestamp)) + + val eventBatches = awaitEventBatchRequest(1000L, 2) + val eventList = eventBatches.flatMap(EventBatch::getEventsList) + assertEquals(4, eventList.size) + assertEquals(4, eventList.filter { it.status == SUCCESS }.size) + } + + @Test + fun `failed verification with expired message timeout`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(SESSION_ALIAS, Direction.FIRST, listOf( + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setTimestamp(getMessageTimestamp(checkpointTimestamp, 600)) + .setId(MessageID.newBuilder().setSequence(1L)) + .build()) + .build() + )) + + val eventID = createEvent("root") + val filter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMetadataFilter(MetadataFilter.newBuilder() + .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) + .build() + val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(500, 1000)) + task.begin(createCheckpoint(checkpointTimestamp)) + + val eventBatches = awaitEventBatchRequest(1000L, 2) + val eventList = eventBatches.flatMap(EventBatch::getEventsList) + assertEquals(3, eventList.size) + assertEquals(2, eventList.filter { it.status == FAILED }.size) + } } \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt new file mode 100644 index 00000000..1947b20e --- /dev/null +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt @@ -0,0 +1,191 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.rule.sequence + +import com.exactpro.th2.check1.SessionKey +import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout +import com.exactpro.th2.check1.grpc.PreFilter +import com.exactpro.th2.check1.rule.AbstractCheckTaskTest +import com.exactpro.th2.common.grpc.Direction +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.EventStatus +import com.exactpro.th2.common.grpc.FilterOperation +import com.exactpro.th2.common.grpc.Message +import com.exactpro.th2.common.grpc.Value +import com.exactpro.th2.common.grpc.ValueFilter +import com.google.protobuf.Timestamp +import io.reactivex.Observable +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertAll +import java.time.Instant +import kotlin.test.assertEquals +import kotlin.test.assertTrue + +class TestNoMessageCheckTask : AbstractCheckTaskTest() { + @Test + fun `no messages outside the prefilter`() { + val checkpointTimestamp = Instant.now() + val messageTimeout = 1500L + val streams = createStreams( + messages = createMessages( + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1300)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1500)), + MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 1600)), + // should be skipped because of message timeout + MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 1600)) + ) + ) + + val eventID = createEvent("root") + val task = noMessageCheckTask( + eventID, + streams, + createPreFilter("A", "1", FilterOperation.EQUAL), + TaskTimeout(messageTimeout, 5000) + ) + task.begin(createCheckpoint(checkpointTimestamp)) + + val eventBatch = awaitEventBatchRequest(1000L, 2) + val eventsList = eventBatch.flatMap(EventBatch::getEventsList) + + assertAll({ + assertTrue(eventsList.all { it.status == EventStatus.SUCCESS }, "Has messages outside the prefilter") + assertTrue(eventsList.first().attachedMessageIdsCount == 5) + assertTrue(eventsList[1].attachedMessageIdsCount == 4) + assertTrue(eventsList.last().attachedMessageIdsCount == 0) + }) + } + + @Test + fun `with messages outside the prefilter`() { + val checkpointTimestamp = Instant.now() + val messageTimeout = 1500L + val streams = createStreams( + messages = createMessages( + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("C", createValue("3"), getMessageTimestamp(checkpointTimestamp, 700)), + MessageData("D", createValue("4"), getMessageTimestamp(checkpointTimestamp, 1600)), + // should be skipped because of message timeout + MessageData("E", createValue("5"), getMessageTimestamp(checkpointTimestamp, 1700)) + ) + ) + + val eventID = createEvent("root") + val task = noMessageCheckTask( + eventID, + streams, + createPreFilter("A", "1", FilterOperation.EQUAL), + TaskTimeout(messageTimeout, 5000) + ) + task.begin(createCheckpoint(checkpointTimestamp)) + + val eventBatch = awaitEventBatchRequest(1000L, 2) + val eventsList = eventBatch.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = eventsList.first() + assertEquals(rootEvent.status, EventStatus.FAILED, "Event status should be failed") + assertTrue(rootEvent.attachedMessageIdsCount == 4) + assertTrue(eventsList[1].attachedMessageIdsCount == 1) + assertTrue(eventsList.last().attachedMessageIdsCount == 2) + }) + } + + @Test + fun `check messages without message timeout`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams( + messages = createMessages( + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 700)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1300)) + ) + ) + + val eventID = createEvent("root") + val task = noMessageCheckTask( + eventID, + streams, + createPreFilter("A", "1", FilterOperation.EQUAL), + TaskTimeout(2000) + ) + task.begin(createCheckpoint(checkpointTimestamp)) + + val eventBatch = awaitEventBatchRequest(1000L, 2) + val eventsList = eventBatch.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = eventsList.first() + assertEquals(rootEvent.status, EventStatus.SUCCESS, "All events should be passed the by prefilter and message timeout") + assertTrue(rootEvent.attachedMessageIdsCount == 5) + assertTrue(eventsList[1].attachedMessageIdsCount == 5) + assertTrue(eventsList.last().attachedMessageIdsCount == 0) + }) + } + + + private fun createMessages( + vararg messageData: MessageData, + sessionAlias: String = SESSION_ALIAS, + messageType: String = MESSAGE_TYPE, + direction: Direction = Direction.FIRST + ): List { + var sequence = 1L; + val messages: MutableList = ArrayList() + messageData.forEach { data -> + messages.add( + constructMessage(sequence++, sessionAlias, messageType, direction, data.timestamp) + .putFields(data.fieldName, data.value) + .build() + ) + } + return messages + } + + private fun createValue(value: String): Value = Value.newBuilder().setSimpleValue(value).build() + + private fun createPreFilter(fieldName: String, value: String, operation: FilterOperation): PreFilter = + PreFilter.newBuilder() + .putFields(fieldName, ValueFilter.newBuilder().setSimpleFilter(value).setKey(true).setOperation(operation).build()) + .build() + + private fun noMessageCheckTask( + parentEventID: EventID, + messageStream: Observable, + preFilterParam: PreFilter, + taskTimeout: TaskTimeout = TaskTimeout(3500,5000L), + maxEventBatchContentSize: Int = 1024 * 1024 + ): NoMessageCheckTask { + return NoMessageCheckTask( + description = "Test", + startTime = Instant.now(), + sessionKey = SessionKey(SESSION_ALIAS, Direction.FIRST), + taskTimeout = taskTimeout, + maxEventBatchContentSize = maxEventBatchContentSize, + protoPreFilter = preFilterParam, + parentEventID = parentEventID, + messageStream = messageStream, + eventBatchRouter = clientStub + ) + } + + data class MessageData(val fieldName: String, val value: Value, val timestamp: Timestamp?) +} \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index c12d5df2..024a929b 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -14,6 +14,7 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTaskTest import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask.Companion.CHECK_MESSAGES_TYPE @@ -107,7 +108,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesInCorrectOrder) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, checkOrder).begin() @@ -168,7 +169,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesUnordered) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, true).begin() @@ -228,7 +229,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesWithKeyFields) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, checkOrder, filtersParam = messageFilters).begin() @@ -294,7 +295,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesWithKeyFields) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, true, filtersParam = messageFilters).begin() @@ -316,6 +317,148 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { }) } + @Test + fun `check sequence of messages with the same value of key field and message timeout`() { + val checkpointTimestamp = Instant.now() + val messagesWithKeyFields: List = listOf( + constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 100)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(2, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 200)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(3, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 300)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build() + ) + + val messageFilter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMessageFilter( + MessageFilter.newBuilder() + .putAllFields( + mapOf( + "A" to ValueFilter.newBuilder().setKey(true).setSimpleFilter("42").build(), + "B" to ValueFilter.newBuilder().setSimpleFilter("AAA").build() + ) + ) + ).build() + val messageFilters: List = listOf( + RootMessageFilter.newBuilder(messageFilter).build(), + RootMessageFilter.newBuilder(messageFilter).build(), + RootMessageFilter.newBuilder(messageFilter).build() + ) + + val messages = Observable.fromIterable(messagesWithKeyFields) + + val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) + val parentEventID = createEvent(EventUtils.generateUUID()) + + sequenceCheckRuleTask( + parentEventID, + messageStream, + true, + filtersParam = messageFilters, + taskTimeout = TaskTimeout(500L, 5000L) + ).begin(createCheckpoint(checkpointTimestamp)) + + val batchRequest = awaitEventBatchRequest(1000L, 6) + val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = assertNotNull(eventsList.find { it.parentId == parentEventID }) + assertEquals(3, rootEvent.attachedMessageIdsCount) + assertEquals(listOf(1L, 2L, 3L), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val checkedMessages = assertNotNull(eventsList.find { it.type == CHECK_MESSAGES_TYPE }, "Cannot find checkMessages event") + val verifications = eventsList.filter { it.parentId == checkedMessages.id } + assertEquals(3, verifications.size, "Unexpected verifications count: $verifications") + assertTrue("Some verifications are not success: $verifications") { verifications.all { it.status == EventStatus.SUCCESS } } + assertEquals(listOf(1L, 2L, 3L), verifications.flatMap { verification -> verification.attachedMessageIdsList.map { it.sequence } }) + }, { + assertCheckSequenceStatus(EventStatus.SUCCESS, eventsList) // because all key fields are in a correct order + }) + } + + @Test + fun `check sequence of messages with the same value of key field and expired message timeout`() { + val checkpointTimestamp = Instant.now() + val messagesWithKeyFields: List = listOf( + constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 500)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(2, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 600)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(3, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 700)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build() + ) + + val messageFilter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMessageFilter( + MessageFilter.newBuilder() + .putAllFields( + mapOf( + "A" to ValueFilter.newBuilder().setKey(true).setSimpleFilter("42").build(), + "B" to ValueFilter.newBuilder().setSimpleFilter("AAA").build() + ) + ) + ).build() + val messageFilters: List = listOf( + RootMessageFilter.newBuilder(messageFilter).build() + ) + + val messages = Observable.fromIterable(messagesWithKeyFields) + + val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) + val parentEventID = createEvent(EventUtils.generateUUID()) + + sequenceCheckRuleTask( + parentEventID, + messageStream, + true, + filtersParam = messageFilters, + taskTimeout = TaskTimeout(500L, 5000L) + ).begin(createCheckpoint(checkpointTimestamp)) + + val batchRequest = awaitEventBatchRequest(1000L, 6) + val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = assertNotNull(eventsList.find { it.parentId == parentEventID }) + assertEquals(1, rootEvent.attachedMessageIdsCount) + assertEquals(listOf(1L), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val checkedMessages = assertNotNull(eventsList.find { it.type == CHECK_MESSAGES_TYPE }, "Cannot find checkMessages event") + val verifications = eventsList.filter { it.parentId == checkedMessages.id } + assertEquals(1, verifications.size, "Unexpected verifications count: $verifications") + assertTrue("Some verifications are not success: $verifications") { verifications.all { it.status == EventStatus.SUCCESS } } + assertEquals(listOf(1L), verifications.flatMap { verification -> verification.attachedMessageIdsList.map { it.sequence } }) + }, { + assertCheckSequenceStatus(EventStatus.SUCCESS, eventsList) // because all key fields are in a correct order + }) + } + @Test fun `check ordering is not failed in case key fields are matches the order but the rest are not`() { val messagesWithKeyFields: List = listOf( @@ -342,7 +485,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesWithKeyFields) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, true).begin() @@ -375,7 +518,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesUnordered) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, false).begin() @@ -429,7 +572,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messages = Observable.fromIterable(messagesWithKeyFields) val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) - val parentEventID = EventID.newBuilder().setId(EventUtils.generateUUID()).build() + val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask(parentEventID, messageStream, false).begin() @@ -462,13 +605,14 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { checkOrder: Boolean, preFilterParam: PreFilter = preFilter, filtersParam: List = protoMessageFilters, - maxEventBatchContentSize: Int = 1024 * 1024 + maxEventBatchContentSize: Int = 1024 * 1024, + taskTimeout: TaskTimeout = TaskTimeout(5000L) ): SequenceCheckRuleTask { return SequenceCheckRuleTask( description = "Test", startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, Direction.FIRST), - timeout = 5000L, + taskTimeout = taskTimeout, maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = filtersParam, From aebe74a8912a81d198ed900e60f91bb49ab20a91 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Wed, 11 Aug 2021 20:45:04 +0400 Subject: [PATCH 04/41] [TH2-1950] Code improvements --- build.gradle | 2 - .../th2/check1/CheckpointSubscriber.kt | 2 +- .../exactpro/th2/check1/CollectorService.kt | 6 +- .../th2/check1/entities/Checkpoint.kt | 27 ++++++--- .../th2/check1/entities/CheckpointData.kt | 13 +++++ .../th2/check1/entities/TaskTimeout.kt | 17 +++++- .../th2/check1/rule/AbstractCheckTask.kt | 2 +- .../rule/sequence/NoMessageCheckTask.kt | 2 +- .../th2/check1/utils/ProtoMessageUtils.kt | 24 ++++---- .../th2/check1/rule/AbstractCheckTaskTest.kt | 4 +- .../com/exactpro/th2/check1/rule/TestChain.kt | 2 +- .../check1/rule/check/TestCheckRuleTask.kt | 4 +- .../rule/sequence/TestNoMessageCheckTask.kt | 55 +++++++++---------- .../rule/sequence/TestSequenceCheckTask.kt | 4 +- 14 files changed, 97 insertions(+), 67 deletions(-) diff --git a/build.gradle b/build.gradle index b282131b..0ed79d39 100644 --- a/build.gradle +++ b/build.gradle @@ -173,8 +173,6 @@ dependencies { implementation "com.exactpro.sf:sailfish-core:${sailfishVersion}" - implementation "com.datastax.cassandra:cassandra-driver-core" //FIXME: Migrate to another library for UUID - implementation "io.reactivex.rxjava2:rxjava:2.2.19" // https://github.com/salesforce/reactive-grpc/issues/202 testImplementation 'org.junit.jupiter:junit-jupiter:5.6.2' diff --git a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt index 786b5f86..1136ef6c 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CheckpointSubscriber.kt @@ -25,7 +25,7 @@ class CheckpointSubscriber : AbstractSessionObserver() { } fun createCheckpoint(): Checkpoint = Checkpoint( - sessionSet.associateBy( + sessionKeyToCheckpointData = sessionSet.associateBy( { session -> session.sessionKey }, { session -> session.lastMessage.metadata.run { diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index f85ab63a..21faca6e 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -90,7 +90,7 @@ class CollectorService( val chainID = request.getChainIdOrGenerate() val task = CheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), - TaskTimeout(request.messageTimeout, request.timeout), maxEventBatchContentSize, + TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, filter, parentEventID, streamObservable, eventBatchRouter) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -123,7 +123,7 @@ class CollectorService( } val task = SequenceCheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), - TaskTimeout(request.messageTimeout, request.timeout), maxEventBatchContentSize, request.preFilter, + TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, protoMessageFilters, request.checkOrder, parentEventID, streamObservable, eventBatchRouter) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -146,7 +146,7 @@ class CollectorService( request.description, Instant.now(), SessionKey(sessionAlias, direction), - TaskTimeout(request.messageTimeout, request.timeout), + TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, parentEventID, diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt index fe538260..27f3891f 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt @@ -1,12 +1,23 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + package com.exactpro.th2.check1.entities -import com.datastax.driver.core.utils.UUIDs.timeBased import com.exactpro.th2.check1.SessionKey +import com.exactpro.th2.common.event.EventUtils + -data class Checkpoint( - val id: String = timeBased().toString(), val sessionKeyToCheckpointData: Map -) { - constructor(sessionKeyToCheckpointData: Map) : this( - timeBased().toString(), sessionKeyToCheckpointData - ) -} \ No newline at end of file +data class Checkpoint @JvmOverloads constructor( + val id: String = EventUtils.generateUUID(), + val sessionKeyToCheckpointData: Map +) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt index 5225af3e..fcb501b3 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt @@ -1,3 +1,16 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + package com.exactpro.th2.check1.entities import com.google.protobuf.Timestamp diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt index 8be40f32..fee55d99 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt @@ -1,5 +1,16 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + package com.exactpro.th2.check1.entities -data class TaskTimeout(val messageTimeout: Long? = null, val timeout: Long) { - constructor(timeout: Long) : this(null, timeout) -} \ No newline at end of file +data class TaskTimeout(val timeout: Long, val messageTimeout: Long? = null) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index d2a6b590..9b776506 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -216,7 +216,7 @@ abstract class AbstractCheckTask( if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } - LOGGER.info("Check begin for session alias '{}' with sequence '{}' timeout '{}'", sessionKey, sequence, taskTimeout.timeout) + LOGGER.info("Check begin for session alias '{}' with sequence '{}' and task timeout '{}'", sessionKey, sequence, taskTimeout) this.lastSequence = sequence this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) this.executorService = executorService diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt index ace89c52..aa85f257 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt @@ -95,7 +95,7 @@ class NoMessageCheckTask( matchNames = false, significant = false ) - messageContainer.protoMessage.metadata.run { + messageContainer.protoMessage.metadata.apply { if (FilterUtils.allMatches(result, protoPreMessageFilter) { it.fullMatch }) { preFilterMessagesCounter++ preFilterEvent.messageID(id) diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt index 8c752a31..2c1a0778 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt @@ -15,7 +15,6 @@ package com.exactpro.th2.check1.utils import com.exactpro.sf.common.messages.IMessage import com.exactpro.th2.check1.SessionKey -import com.exactpro.th2.check1.entities.CheckpointData import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.common.grpc.Checkpoint @@ -24,6 +23,9 @@ import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter +import com.exactpro.th2.check1.entities.Checkpoint as InternalCheckpoint +import com.exactpro.th2.check1.entities.CheckpointData as InternalCheckpointData +import com.exactpro.th2.common.grpc.Checkpoint.CheckpointData as CheckpointData fun ProtoToIMessageConverter.fromProtoPreFilter(protoPreMessageFilter: RootMessageFilter): IMessage = fromProtoFilter(protoPreMessageFilter.messageFilter, SequenceCheckRuleTask.PRE_FILTER_MESSAGE_NAME) @@ -42,16 +44,16 @@ fun PreFilter.toMessageFilter(): MessageFilter = MessageFilter.newBuilder() .putAllFields(fieldsMap) .build() -fun Checkpoint.CheckpointData.convert(): CheckpointData = CheckpointData(sequence, timestamp) +fun CheckpointData.convert(): InternalCheckpointData = InternalCheckpointData(sequence, timestamp) -fun CheckpointData.convert(): Checkpoint.CheckpointData { - val builder = Checkpoint.CheckpointData.newBuilder().setSequence(sequence) +fun InternalCheckpointData.convert(): CheckpointData { + val builder = CheckpointData.newBuilder().setSequence(sequence) if (timestamp != null) builder.timestamp = timestamp return builder.build() } -fun com.exactpro.th2.check1.entities.Checkpoint.convert(): Checkpoint { +fun InternalCheckpoint.convert(): Checkpoint { val intermediateMap: MutableMap = HashMap() sessionKeyToCheckpointData.forEach { (sessionKey, checkpointData) -> intermediateMap.computeIfAbsent(sessionKey.sessionAlias) { @@ -67,16 +69,16 @@ fun com.exactpro.th2.check1.entities.Checkpoint.convert(): Checkpoint { return checkpointBuilder.build() } -fun Checkpoint.convert(): com.exactpro.th2.check1.entities.Checkpoint { - val sessionKeyToSequence: MutableMap = HashMap() +fun Checkpoint.convert(): InternalCheckpoint { + val sessionKeyToSequence: MutableMap = HashMap() sessionAliasToDirectionCheckpointMap.forEach { (sessionAlias, directionCheckpoint) -> - check(!(directionCheckpoint.directionToCheckpointDataCount != 0 && directionCheckpoint.directionToSequenceCount != 0)) { + check(!(directionCheckpoint.run { directionToCheckpointDataCount != 0 && directionToSequenceCount != 0 })) { "Session alias '${sessionAlias}' cannot contain both of these fields: 'direction to checkpoint data' and 'direction to sequence'. Please use 'direction to checkpoint data' instead" } if (directionCheckpoint.directionToCheckpointDataCount == 0) { - directionCheckpoint.directionToSequenceMap.forEach { (directionNumber, sequence) -> + directionCheckpoint.directionToSequenceMap.forEach { (directionNumber, sequence) -> val sessionKey = SessionKey(sessionAlias, Direction.forNumber(directionNumber)) - sessionKeyToSequence[sessionKey] = CheckpointData(sequence, null) + sessionKeyToSequence[sessionKey] = InternalCheckpointData(sequence, null) } } else { directionCheckpoint.directionToCheckpointDataMap.forEach { (directionNumber, checkpointData) -> @@ -86,5 +88,5 @@ fun Checkpoint.convert(): com.exactpro.th2.check1.entities.Checkpoint { } } } - return com.exactpro.th2.check1.entities.Checkpoint(id, sessionKeyToSequence) + return InternalCheckpoint(id, sessionKeyToSequence) } \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index 168b9ce6..c42e1fa0 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -49,11 +49,11 @@ abstract class AbstractCheckTaskTest { alias: String = SESSION_ALIAS, type: String = MESSAGE_TYPE, direction: Direction = FIRST, - timestamp: Timestamp? = Timestamp.getDefaultInstance() + timestamp: Timestamp = Timestamp.getDefaultInstance() ): Message.Builder = Message.newBuilder().apply { metadataBuilder.apply { this.messageType = type - this.timestamp = timestamp ?: Timestamp.getDefaultInstance() + this.timestamp = timestamp idBuilder.apply { this.sequence = sequence this.direction = direction diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index 144ba21d..9945b38f 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -169,7 +169,7 @@ class TestChain: AbstractCheckTaskTest() { description = "Test", startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, FIRST), - taskTimeout = TaskTimeout(null, 1000L), + taskTimeout = TaskTimeout(1000L), maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = sequence.map(::createMessageFilter).toList(), diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index 1becf916..393d15e8 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -214,7 +214,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .setMetadataFilter(MetadataFilter.newBuilder() .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() - val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(500, 1000)) + val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(1000, 500)) task.begin(createCheckpoint(checkpointTimestamp)) val eventBatches = awaitEventBatchRequest(1000L, 2) @@ -243,7 +243,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .setMetadataFilter(MetadataFilter.newBuilder() .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() - val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(500, 1000)) + val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(1000, 500)) task.begin(createCheckpoint(checkpointTimestamp)) val eventBatches = awaitEventBatchRequest(1000L, 2) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt index 1947b20e..e3f78261 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt @@ -26,6 +26,7 @@ import com.exactpro.th2.common.grpc.FilterOperation import com.exactpro.th2.common.grpc.Message import com.exactpro.th2.common.grpc.Value import com.exactpro.th2.common.grpc.ValueFilter +import com.exactpro.th2.common.value.toValue import com.google.protobuf.Timestamp import io.reactivex.Observable import org.junit.jupiter.api.Test @@ -41,13 +42,13 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val messageTimeout = 1500L val streams = createStreams( messages = createMessages( - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1000)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1300)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1500)), - MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 1600)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), + MessageData("B", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), // should be skipped because of message timeout - MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 1600)) + MessageData("B", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) ) ) @@ -56,7 +57,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { eventID, streams, createPreFilter("A", "1", FilterOperation.EQUAL), - TaskTimeout(messageTimeout, 5000) + TaskTimeout(5000, messageTimeout) ) task.begin(createCheckpoint(checkpointTimestamp)) @@ -77,12 +78,12 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val messageTimeout = 1500L val streams = createStreams( messages = createMessages( - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 100)), - MessageData("B", createValue("2"), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("C", createValue("3"), getMessageTimestamp(checkpointTimestamp, 700)), - MessageData("D", createValue("4"), getMessageTimestamp(checkpointTimestamp, 1600)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 700)), + MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), // should be skipped because of message timeout - MessageData("E", createValue("5"), getMessageTimestamp(checkpointTimestamp, 1700)) + MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1700)) ) ) @@ -91,7 +92,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { eventID, streams, createPreFilter("A", "1", FilterOperation.EQUAL), - TaskTimeout(messageTimeout, 5000) + TaskTimeout(5000, messageTimeout) ) task.begin(createCheckpoint(checkpointTimestamp)) @@ -112,11 +113,11 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val checkpointTimestamp = Instant.now() val streams = createStreams( messages = createMessages( - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 100)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 700)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1000)), - MessageData("A", createValue("1"), getMessageTimestamp(checkpointTimestamp, 1300)) + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 700)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)) ) ) @@ -149,19 +150,13 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { direction: Direction = Direction.FIRST ): List { var sequence = 1L; - val messages: MutableList = ArrayList() - messageData.forEach { data -> - messages.add( - constructMessage(sequence++, sessionAlias, messageType, direction, data.timestamp) - .putFields(data.fieldName, data.value) - .build() - ) + return messageData.map { data -> + constructMessage(sequence++, sessionAlias, messageType, direction, data.timestamp) + .putFields(data.fieldName, data.value) + .build() } - return messages } - private fun createValue(value: String): Value = Value.newBuilder().setSimpleValue(value).build() - private fun createPreFilter(fieldName: String, value: String, operation: FilterOperation): PreFilter = PreFilter.newBuilder() .putFields(fieldName, ValueFilter.newBuilder().setSimpleFilter(value).setKey(true).setOperation(operation).build()) @@ -171,7 +166,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { parentEventID: EventID, messageStream: Observable, preFilterParam: PreFilter, - taskTimeout: TaskTimeout = TaskTimeout(3500,5000L), + taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L), maxEventBatchContentSize: Int = 1024 * 1024 ): NoMessageCheckTask { return NoMessageCheckTask( @@ -187,5 +182,5 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { ) } - data class MessageData(val fieldName: String, val value: Value, val timestamp: Timestamp?) + data class MessageData(val fieldName: String, val value: Value, val timestamp: Timestamp) } \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index 024a929b..29afa333 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -368,7 +368,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { messageStream, true, filtersParam = messageFilters, - taskTimeout = TaskTimeout(500L, 5000L) + taskTimeout = TaskTimeout(5000L, 500L) ).begin(createCheckpoint(checkpointTimestamp)) val batchRequest = awaitEventBatchRequest(1000L, 6) @@ -438,7 +438,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { messageStream, true, filtersParam = messageFilters, - taskTimeout = TaskTimeout(500L, 5000L) + taskTimeout = TaskTimeout(5000L, 500L) ).begin(createCheckpoint(checkpointTimestamp)) val batchRequest = awaitEventBatchRequest(1000L, 6) From 1ef18a311a63eeaa8293368ec4d23c6d6bc94348 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 12 Aug 2021 16:20:11 +0400 Subject: [PATCH 05/41] [TH2-1950] Updated stream completion cases * Move NoMessageCheckTask to its own package * Corrected NoMessageCheckTask logic --- .../exactpro/th2/check1/CollectorService.kt | 2 +- .../th2/check1/rule/AbstractCheckTask.kt | 62 +++++++++++++------ .../NoMessageCheckTask.kt | 12 ++-- .../rule/sequence/SequenceCheckRuleTask.kt | 2 +- .../th2/check1/utils/ProtoMessageUtils.kt | 2 +- .../TestNoMessageCheckTask.kt | 20 +++--- 6 files changed, 64 insertions(+), 36 deletions(-) rename src/main/kotlin/com/exactpro/th2/check1/rule/{sequence => nomessage}/NoMessageCheckTask.kt (96%) rename src/test/kotlin/com/exactpro/th2/check1/rule/{sequence => nomessage}/TestNoMessageCheckTask.kt (93%) diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 21faca6e..231b63d9 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -19,7 +19,7 @@ import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.* import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.check.CheckRuleTask -import com.exactpro.th2.check1.rule.sequence.NoMessageCheckTask +import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.EventUtils diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 9b776506..5098c5e1 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -100,8 +100,11 @@ abstract class AbstractCheckTask( CREATED, BEGIN, TIMEOUT, - COMPLETED, - PUBLISHED + MESSAGE_TIMEOUT, + TASK_COMPLETED, + STREAM_COMPLETED, + PUBLISHED, + ERROR } @Volatile @@ -122,7 +125,7 @@ abstract class AbstractCheckTask( rootEvent.status(FAILED) .bodyData(EventUtils.createMessageBean(e.message)) - end("Error ${e.message} received in message stream") + end(State.ERROR, "Error ${e.message} received in message stream") } /** @@ -181,20 +184,30 @@ abstract class AbstractCheckTask( protected open fun onTimeout() {} /** - * Marks the task as successfully completed. If the task timeout had been exited and then the task was marked as successfully completed - * the task will be considered as successfully completed because it had actually found that it should + * Marks the task as successfully completed. If the task timeout, message timeout or stream had been exited and then + * the task was marked as successfully completed the task will be considered as successfully completed because + * it had actually found that it should */ protected fun checkComplete() { LOGGER.info("Check completed for session alias '{}' with sequence '{}'", sessionKey, lastSequence) - val prevValue = taskState.getAndSet(State.COMPLETED) + val prevValue = taskState.getAndSet(State.TASK_COMPLETED) dispose() endFuture.dispose() - if (prevValue == State.TIMEOUT) { - LOGGER.info("Task '{}' for session alias '{}' is completed right after timeout exited. Consider it as completed", description, sessionKey) - } else { - LOGGER.debug("Task '{}' for session alias '{}' is completed normally", description, sessionKey) + when (prevValue) { + State.TIMEOUT -> { + LOGGER.info("Task '{}' for session alias '{}' is completed right after timeout exited. Consider it as completed", description, sessionKey) + } + State.MESSAGE_TIMEOUT -> { + LOGGER.info("Task '{}' for session alias '{}' is completed right after message timeout exited. Consider it as completed", description, sessionKey) + } + State.STREAM_COMPLETED -> { + LOGGER.info("Task '{}' for session alias '{}' is completed right after the end of streaming messages. Consider it as completed", description, sessionKey) + } + else -> { + LOGGER.debug("Task '{}' for session alias '{}' is completed normally", description, sessionKey) + } } } @@ -223,7 +236,7 @@ abstract class AbstractCheckTask( val scheduler = Schedulers.from(executorService) endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) - .subscribe { _ -> end("Timeout is exited") } + .subscribe { _ -> end(State.TIMEOUT, "Timeout is exited") } messageStream.observeOn(scheduler) // Defined scheduler to execution in one thread to avoid race-condition. .doFinally(this::taskFinished) // will be executed if the source is complete or an error received or the timeout is exited. @@ -245,7 +258,7 @@ abstract class AbstractCheckTask( rootEvent.messageID(this.id) } } - .takeWhile { checkOnMessageTimeout(it.metadata.timestamp) } + .onMessageTimeout() .mapToMessageContainer() .taskPipeline() .subscribe(this) @@ -255,7 +268,7 @@ abstract class AbstractCheckTask( try { val currentState = taskState.get() LOGGER.info("Finishes task '$description' in state $currentState") - if (currentState == State.TIMEOUT) { + if (currentState == State.TIMEOUT || currentState == State.MESSAGE_TIMEOUT || currentState == State.STREAM_COMPLETED) { callOnTimeoutCallback() } publishEvent() @@ -292,10 +305,11 @@ abstract class AbstractCheckTask( * Disposes the task when the timeout is over or the message stream is completed normally or with an exception. * Task unsubscribe from the message stream. * + * @param state of the stopped task * @param reason the cause why a task must be stopped */ - private fun end(reason: String) { - if (taskState.compareAndSet(State.BEGIN, State.TIMEOUT)) { + private fun end(state: State, reason: String) { + if (taskState.compareAndSet(State.BEGIN, state)) { LOGGER.info("Stop task for session alias '{}' with sequence '{}' because: {}", sessionKey, lastSequence, reason) dispose() endFuture.dispose() @@ -306,14 +320,14 @@ abstract class AbstractCheckTask( override fun onComplete() { super.onComplete() - end("Message stream is completed") + end(State.STREAM_COMPLETED, "Message stream is completed") } /** * Prepare the root event or children events for publication. * This method is invoked in [State.PUBLISHED] state. */ - protected open fun completeEvent(canceled: Boolean) {} + protected open fun completeEvent(taskState: State) {} /** * Publishes the event to [eventBatchRouter]. @@ -321,7 +335,7 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - completeEvent(prevState == State.TIMEOUT) + completeEvent(prevState) _endTime = Instant.now() val batches = rootEvent.disperseToBatches(maxEventBatchContentSize, parentEventID) @@ -506,6 +520,18 @@ abstract class AbstractCheckTask( return checkpointTimeout == null || checkpointTimeout!!.isAfter(timestamp) || checkpointTimeout == timestamp } + /** + * Provides the ability to stop observing if a message timeout is set. + */ + private fun Observable.onMessageTimeout() : Observable = + takeWhile { + checkOnMessageTimeout(it.metadata.timestamp).also { continueObservation -> + if (!continueObservation) { + end(State.MESSAGE_TIMEOUT, "Expired message timeout") + } + } + } + private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? = if (timestamp == null || messageTimeout == null) { null diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt similarity index 96% rename from src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt rename to src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index aa85f257..283dc9b2 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -11,7 +11,7 @@ * limitations under the License. */ -package com.exactpro.th2.check1.rule.sequence +package com.exactpro.th2.check1.rule.nomessage import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer @@ -97,16 +97,16 @@ class NoMessageCheckTask( ) messageContainer.protoMessage.metadata.apply { if (FilterUtils.allMatches(result, protoPreMessageFilter) { it.fullMatch }) { - preFilterMessagesCounter++ - preFilterEvent.messageID(id) - } else { extraMessagesCounter++ resultEvent.messageID(id) + } else { + preFilterMessagesCounter++ + preFilterEvent.messageID(id) } } } - override fun completeEvent(canceled: Boolean) { + override fun completeEvent(taskState: State) { preFilterEvent.name("Prefilter: $preFilterMessagesCounter messages were filtered.") if (extraMessagesCounter == 0) { @@ -115,5 +115,7 @@ class NoMessageCheckTask( resultEvent.status(Event.Status.FAILED) .name("Check failed: $extraMessagesCounter extra messages were found.") } + + resultEvent.description("Task has been completed because: {$taskState}") } } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 407bf9c6..ab890fab 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -183,7 +183,7 @@ class SequenceCheckRuleTask( } } - override fun completeEvent(canceled: Boolean) { + override fun completeEvent(taskState: State) { preFilterEvent.name("Pre-filtering (filtered ${preFilteringResults.size} / processed $handledMessageCounter) messages") fillSequenceEvent() diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt index 2c1a0778..037f8126 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt @@ -18,6 +18,7 @@ import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.common.grpc.Checkpoint +import com.exactpro.th2.common.grpc.Checkpoint.CheckpointData import com.exactpro.th2.common.grpc.Checkpoint.DirectionCheckpoint import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.MessageFilter @@ -25,7 +26,6 @@ import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter import com.exactpro.th2.check1.entities.Checkpoint as InternalCheckpoint import com.exactpro.th2.check1.entities.CheckpointData as InternalCheckpointData -import com.exactpro.th2.common.grpc.Checkpoint.CheckpointData as CheckpointData fun ProtoToIMessageConverter.fromProtoPreFilter(protoPreMessageFilter: RootMessageFilter): IMessage = fromProtoFilter(protoPreMessageFilter.messageFilter, SequenceCheckRuleTask.PRE_FILTER_MESSAGE_NAME) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt similarity index 93% rename from src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt rename to src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index e3f78261..ae217d08 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -11,7 +11,7 @@ * limitations under the License. */ -package com.exactpro.th2.check1.rule.sequence +package com.exactpro.th2.check1.rule.nomessage import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer @@ -43,12 +43,12 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val streams = createStreams( messages = createMessages( MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), - MessageData("B", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), + MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), + MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), + MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), // should be skipped because of message timeout - MessageData("B", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) + MessageData("F", "6".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) ) ) @@ -56,7 +56,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val task = noMessageCheckTask( eventID, streams, - createPreFilter("A", "1", FilterOperation.EQUAL), + createPreFilter("E", "5", FilterOperation.EQUAL), TaskTimeout(5000, messageTimeout) ) task.begin(createCheckpoint(checkpointTimestamp)) @@ -103,8 +103,8 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val rootEvent = eventsList.first() assertEquals(rootEvent.status, EventStatus.FAILED, "Event status should be failed") assertTrue(rootEvent.attachedMessageIdsCount == 4) - assertTrue(eventsList[1].attachedMessageIdsCount == 1) - assertTrue(eventsList.last().attachedMessageIdsCount == 2) + assertTrue(eventsList[1].attachedMessageIdsCount == 2) + assertTrue(eventsList.last().attachedMessageIdsCount == 1) }) } @@ -125,7 +125,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val task = noMessageCheckTask( eventID, streams, - createPreFilter("A", "1", FilterOperation.EQUAL), + createPreFilter("B", "2", FilterOperation.EQUAL), TaskTimeout(2000) ) task.begin(createCheckpoint(checkpointTimestamp)) From 7def97c8c5f392881c323d2051f193db25ebbc54 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Sat, 14 Aug 2021 16:11:48 +0400 Subject: [PATCH 06/41] [TH2-1984] Updated SequenceCheckRuleTask logic * Updated test cases for new execution rules --- .../th2/check1/rule/AbstractCheckTask.kt | 94 +++++++++--- .../rule/nomessage/NoMessageCheckTask.kt | 6 +- .../rule/sequence/SequenceCheckRuleTask.kt | 40 ++++- .../th2/check1/rule/AbstractCheckTaskTest.kt | 6 +- .../com/exactpro/th2/check1/rule/TestChain.kt | 56 ++++++- .../rule/nomessage/TestNoMessageCheckTask.kt | 9 +- .../rule/sequence/TestSequenceCheckTask.kt | 140 +++++++++++++++--- 7 files changed, 291 insertions(+), 60 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 5098c5e1..9fd12448 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -31,7 +31,14 @@ import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.Event.Status.FAILED import com.exactpro.th2.common.event.Event.Status.PASSED import com.exactpro.th2.common.event.EventUtils -import com.exactpro.th2.common.grpc.* +import com.exactpro.th2.common.grpc.Checkpoint +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.Message +import com.exactpro.th2.common.grpc.MessageFilter +import com.exactpro.th2.common.grpc.MessageMetadata +import com.exactpro.th2.common.grpc.MetadataFilter +import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toTreeTable import com.exactpro.th2.common.schema.message.MessageRouter @@ -84,6 +91,8 @@ abstract class AbstractCheckTask( private val sequenceSubject = SingleSubject.create() private val hasNextTask = AtomicBoolean(false) private val taskState = AtomicReference(State.CREATED) + @Volatile + private var executionState = State.STREAM_COMPLETED /** * Used for observe messages in one thread. @@ -96,15 +105,15 @@ abstract class AbstractCheckTask( val endTime: Instant? get() = _endTime - protected enum class State { - CREATED, - BEGIN, - TIMEOUT, - MESSAGE_TIMEOUT, - TASK_COMPLETED, - STREAM_COMPLETED, - PUBLISHED, - ERROR + protected enum class State(val callOnTimeoutCallback: Boolean) { + CREATED(false), + BEGIN(false), + TIMEOUT(true), + MESSAGE_TIMEOUT(true), + TASK_COMPLETED(false), + STREAM_COMPLETED(true), + PUBLISHED(false), + ERROR(false) } @Volatile @@ -112,6 +121,12 @@ abstract class AbstractCheckTask( private var lastSequence = DEFAULT_SEQUENCE private var checkpointTimeout: Timestamp? = null + private var lastMessageTimestamp: Timestamp? = null + private var untrusted: Boolean = false + protected var hasMessagesInTimeoutInterval: Boolean = false + private set + protected var bufferContainsStartMessage: Boolean = false + private set override fun onStart() { super.onStart() @@ -158,7 +173,9 @@ abstract class AbstractCheckTask( } else { legacy.executorService } - checkTask.begin(legacy.lastSequence, legacy.checkpointTimestamp, executor) + legacy.sequenceData.apply { + checkTask.begin(this.lastSequence, this.lastMessageTimestamp, executor, this.untrusted) + } } LOGGER.info("Task {} ({}) subscribed to task {} ({})", checkTask.description, checkTask.hashCode(), description, hashCode()) } else { @@ -221,18 +238,28 @@ abstract class AbstractCheckTask( * Task subscribe to messages stream with sequence after call. * This method should be called only once otherwise it throws IllegalStateException. * @param sequence message sequence from the previous task. + * @param checkpointTimestamp checkpoint timestamp from the previous task * @param executorService executor to schedule pipeline execution. + * @param untrusted flag is guarantee that the previous sequence data is correct * @throws IllegalStateException when method is called more than once. */ private fun begin(sequence: Long = DEFAULT_SEQUENCE, checkpointTimestamp: Timestamp? = null, - executorService: ExecutorService = createExecutorService()) { + executorService: ExecutorService = createExecutorService(), untrusted: Boolean = false) { if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } LOGGER.info("Check begin for session alias '{}' with sequence '{}' and task timeout '{}'", sessionKey, sequence, taskTimeout) this.lastSequence = sequence - this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) this.executorService = executorService + if (untrusted) { + doOnUntrustedExecution().also { + this.untrusted = untrusted + taskState.set(State.ERROR) + taskFinished() + } + return + } + this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) val scheduler = Schedulers.from(executorService) endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) @@ -258,7 +285,7 @@ abstract class AbstractCheckTask( rootEvent.messageID(this.id) } } - .onMessageTimeout() + .takeWhileMessagesInTimeout() .mapToMessageContainer() .taskPipeline() .subscribe(this) @@ -267,8 +294,8 @@ abstract class AbstractCheckTask( private fun taskFinished() { try { val currentState = taskState.get() - LOGGER.info("Finishes task '$description' in state $currentState") - if (currentState == State.TIMEOUT || currentState == State.MESSAGE_TIMEOUT || currentState == State.STREAM_COMPLETED) { + LOGGER.info("Finishes task '$description' in state ${currentState.name}") + if (currentState.callOnTimeoutCallback) { callOnTimeoutCallback() } publishEvent() @@ -287,7 +314,8 @@ abstract class AbstractCheckTask( .toProto(parentEventID)) .build()) } finally { - sequenceSubject.onSuccess(Legacy(executorService, lastSequence, checkpointTimeout)) + val untrustedForFutureTask = !bufferContainsStartMessage && !hasMessagesInTimeoutInterval + sequenceSubject.onSuccess(Legacy(executorService, SequenceData(lastSequence, lastMessageTimestamp, untrustedForFutureTask))) } } @@ -320,7 +348,7 @@ abstract class AbstractCheckTask( override fun onComplete() { super.onComplete() - end(State.STREAM_COMPLETED, "Message stream is completed") + end(executionState, "Message stream is completed") } /** @@ -335,7 +363,9 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - completeEvent(prevState) + if (!untrusted) { + completeEvent(prevState) + } _endTime = Instant.now() val batches = rootEvent.disperseToBatches(maxEventBatchContentSize, parentEventID) @@ -388,7 +418,10 @@ abstract class AbstractCheckTask( return if (comparisonResult != null || metadataComparisonResult != null) { if (significant) { - lastSequence = messageContainer.protoMessage.metadata.id.sequence + messageContainer.protoMessage.metadata.apply { + lastSequence = id.sequence + lastMessageTimestamp = timestamp + } } AggregatedFilterResult(comparisonResult, metadataComparisonResult) } else { @@ -396,6 +429,12 @@ abstract class AbstractCheckTask( } } + /** + * This method is called before the completion of the task if the untrusted flag is set. + * If the task has the untrusted flag, then [onComplete] will not be called. + */ + protected open fun doOnUntrustedExecution() {} + companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE private val RESPONSE_EXECUTOR = ForkJoinPool.commonPool() @@ -490,7 +529,12 @@ abstract class AbstractCheckTask( private fun Observable.continueObserve(sessionKey: SessionKey, sequence: Long): Observable = filter { streamContainer -> streamContainer.sessionKey == sessionKey } .flatMap(StreamContainer::bufferedMessages) - .filter { message -> message.metadata.id.sequence > sequence } + .filter { message -> + if (message.metadata.id.sequence == sequence) { + bufferContainsStartMessage = true + } + message.metadata.id.sequence > sequence + } private fun Checkpoint.getCheckpointData(sessionKey: SessionKey): CheckpointData { val checkpointData = sessionAliasToDirectionCheckpointMap[sessionKey.sessionAlias] @@ -523,11 +567,12 @@ abstract class AbstractCheckTask( /** * Provides the ability to stop observing if a message timeout is set. */ - private fun Observable.onMessageTimeout() : Observable = + private fun Observable.takeWhileMessagesInTimeout() : Observable = takeWhile { checkOnMessageTimeout(it.metadata.timestamp).also { continueObservation -> + hasMessagesInTimeoutInterval = hasMessagesInTimeoutInterval or continueObservation if (!continueObservation) { - end(State.MESSAGE_TIMEOUT, "Expired message timeout") + executionState = State.MESSAGE_TIMEOUT } } } @@ -539,5 +584,6 @@ abstract class AbstractCheckTask( Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) } - private data class Legacy(val executorService: ExecutorService, val lastSequence: Long, val checkpointTimestamp: Timestamp?) + private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) + private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 283dc9b2..efec0f16 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -116,6 +116,10 @@ class NoMessageCheckTask( .name("Check failed: $extraMessagesCounter extra messages were found.") } - resultEvent.description("Task has been completed because: {$taskState}") + if (taskState == State.TIMEOUT || taskState == State.STREAM_COMPLETED) { + resultEvent.addSubEvent( + Event.start().name("Task has been completed because: ${taskState.name}").status(Event.Status.FAILED) + ) + } } } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index ab890fab..3d74091d 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -185,9 +185,18 @@ class SequenceCheckRuleTask( override fun completeEvent(taskState: State) { preFilterEvent.name("Pre-filtering (filtered ${preFilteringResults.size} / processed $handledMessageCounter) messages") + fillEvents { + fillSequenceEvent() + fillCheckMessagesEvent() + } + } - fillSequenceEvent() - fillCheckMessagesEvent() + override fun doOnUntrustedExecution() { + rootEvent.addSubEvent( + Event.start() + .name("The current check is untrusted because the start point of the check interval has been selected approximately") + .status(FAILED) + ) } /** @@ -239,6 +248,33 @@ class SequenceCheckRuleTask( .bodyData(sequenceTable.build()) } + private fun fillMissedStartMessageAndMessagesInIntervalEvent() { + rootEvent.addSubEvent( + Event.start() + .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval") + .status(FAILED) + ) + } + + private fun fillEmptyStarMessageEvent() { + rootEvent.addSubEvent( + Event.start() + .name("Buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' doesn't contain starting message, but contains several messages in the requested check interval") + .status(FAILED) + ) + } + + private fun fillEvents(commonEvents: () -> Unit) { + if (!bufferContainsStartMessage && !hasMessagesInTimeoutInterval) { + fillMissedStartMessageAndMessagesInIntervalEvent() + } else if (!bufferContainsStartMessage) { + commonEvents() + fillEmptyStarMessageEvent() + } else { + commonEvents() + } + } + companion object { const val PRE_FILTER_MESSAGE_NAME = "PreFilter" const val CHECK_MESSAGES_TYPE = "checkMessages" diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index c42e1fa0..ebe5c56c 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -69,7 +69,7 @@ abstract class AbstractCheckTaskTest { protected fun getMessageTimestamp(start: Instant, delta: Long): Timestamp = start.plusMillis(delta).toTimestamp() - protected fun createCheckpoint(timestamp: Instant, sequence: Long = -1) : com.exactpro.th2.common.grpc.Checkpoint = + protected fun createCheckpoint(timestamp: Instant? = null, sequence: Long = -1) : com.exactpro.th2.common.grpc.Checkpoint = com.exactpro.th2.common.grpc.Checkpoint.newBuilder().apply { putSessionAliasToDirectionCheckpoint( SESSION_ALIAS, @@ -78,7 +78,9 @@ abstract class AbstractCheckTaskTest { FIRST.number, com.exactpro.th2.common.grpc.Checkpoint.CheckpointData.newBuilder().apply { this.sequence = sequence - this.timestamp = timestamp.toTimestamp() + if (timestamp != null) { + this.timestamp = timestamp.toTimestamp() + } }.build() ) }.build() diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index 9945b38f..4aba748c 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -33,8 +33,10 @@ import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.value.toValue import io.reactivex.Observable import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertAll import java.time.Instant import kotlin.test.assertEquals +import kotlin.test.assertTrue class TestChain: AbstractCheckTaskTest() { @@ -71,9 +73,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - two succeed`() { - val streams = createStreams(messages = (1..4).map(::createMessage)) + val streams = createStreams(messages = (0..4).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(1, 2), eventID, streams).also { it.begin() } + val task = sequenceCheckRuleTask(listOf(1, 2), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } var eventList = awaitEventBatchAndGetEvents(6, 6) checkSequenceVerifySuccess(eventList, listOf(1, 2)) @@ -84,9 +86,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - full failed, succeed`() { - val streams = createStreams(messages = (1..2).map(::createMessage)) + val streams = createStreams(messages = (0..2).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(3, 4), eventID, streams).also { it.begin() } + val task = sequenceCheckRuleTask(listOf(3, 4), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } var eventList = awaitEventBatchAndGetEvents(6, 6) assertEquals(8, eventList.size) assertEquals(4, eventList.filter { it.status == SUCCESS }.size) @@ -99,9 +101,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - part failed, succeed`() { - val streams = createStreams(messages = (1..3).map(::createMessage)) + val streams = createStreams(messages = (0..3).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(1, 4), eventID, streams).also { it.begin() } + val task = sequenceCheckRuleTask(listOf(1, 4), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } var eventList = awaitEventBatchAndGetEvents(6, 6) assertEquals(9, eventList.size) assertEquals(5, eventList.filter { it.status == SUCCESS }.size) @@ -132,6 +134,43 @@ class TestChain: AbstractCheckTaskTest() { assertEquals(listOf(1L, 2L, 3L, 4L), eventList.filter { it.type == VERIFICATION_TYPE }.flatMap(Event::getAttachedMessageIdsList).map(MessageID::getSequence)) } + @Test + fun `sequence rules - untrusted execution`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(messages = (1..5L).map { + constructMessage(it, timestamp = getMessageTimestamp(checkpointTimestamp, it * 1000)) + .putAllFields( + mapOf( + KEY_FIELD to "$KEY_FIELD$it".toValue(), + NOT_KEY_FIELD to "$NOT_KEY_FIELD$it".toValue() + ) + ).build() + }) + + val task = sequenceCheckRuleTask( + listOf(1, 2), + eventID, + streams, + taskTimeout = TaskTimeout(2000L, 500) + ).also { it.begin(createCheckpoint(checkpointTimestamp, 0)) } + var eventsList = awaitEventBatchAndGetEvents(2, 2) + assertAll({ + val rootEvent = eventsList.first() + assertEquals(FAILED, rootEvent.status, "Event status should be failed") + assertTrue(rootEvent.attachedMessageIdsCount == 1) + }) + + sequenceCheckRuleTask( + listOf(3, 4), + eventID, + streams, + taskTimeout = TaskTimeout(2000L, 1500L) + ).also { task.subscribeNextTask(it) } + eventsList = awaitEventBatchAndGetEvents(4, 4) + assertEquals("The current check is untrusted because the start point of the check interval has been selected approximately", eventsList.last().name) + } + + private fun awaitEventBatchAndGetEvents(times: Int, last: Int): List = awaitEventBatchRequest(1000L, times).drop(times - last).flatMap(EventBatch::getEventsList) @@ -163,13 +202,14 @@ class TestChain: AbstractCheckTaskTest() { messageStream: Observable, checkOrder: Boolean = true, preFilterParam: PreFilter = preFilter, - maxEventBatchContentSize: Int = 1024 * 1024 + maxEventBatchContentSize: Int = 1024 * 1024, + taskTimeout: TaskTimeout = TaskTimeout(1000L) ): SequenceCheckRuleTask { return SequenceCheckRuleTask( description = "Test", startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, FIRST), - taskTimeout = TaskTimeout(1000L), + taskTimeout = taskTimeout, maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = sequence.map(::createMessageFilter).toList(), diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index ae217d08..a3f2edc4 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -130,15 +130,16 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { ) task.begin(createCheckpoint(checkpointTimestamp)) - val eventBatch = awaitEventBatchRequest(1000L, 2) + val eventBatch = awaitEventBatchRequest(1000L, 4) val eventsList = eventBatch.flatMap(EventBatch::getEventsList) assertAll({ val rootEvent = eventsList.first() - assertEquals(rootEvent.status, EventStatus.SUCCESS, "All events should be passed the by prefilter and message timeout") + assertEquals(rootEvent.status, EventStatus.FAILED, "Root event should be failed due to timeout") assertTrue(rootEvent.attachedMessageIdsCount == 5) - assertTrue(eventsList[1].attachedMessageIdsCount == 5) - assertTrue(eventsList.last().attachedMessageIdsCount == 0) + assertTrue(eventsList[1].attachedMessageIdsCount == 0) + assertTrue(eventsList[1].name == "Check passed", "All messages should be ignored due to prefilter") + assertTrue(eventsList[3].attachedMessageIdsCount == 5) }) } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index 29afa333..19c05a5d 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -78,6 +78,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { ) private val messagesInCorrectOrder: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("1").build(), + "B" to Value.newBuilder().setSimpleValue("A").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -110,7 +116,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, checkOrder).begin() + sequenceCheckRuleTask(parentEventID, messageStream, checkOrder).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -166,12 +172,11 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { set(indexesToSwitch.first, get(indexesToSwitch.second)) set(indexesToSwitch.second, tmp) } - val messages = Observable.fromIterable(messagesUnordered) - val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) + val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesUnordered) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true).begin() + sequenceCheckRuleTask(parentEventID, messageStream, true).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -186,7 +191,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { assertEquals(3, passedVerifications.size, "Unexpected SUCCESS verifications count: $passedVerifications") assertTrue("Some verifications have more than one message attached") { passedVerifications.all { it.attachedMessageIdsCount == 1 } } // Ids in the result of the rule are in order by filters because the rule creates events related to verifications/filters in the source order. - assertEquals(messagesInCorrectOrder.map { it.metadata.id }, passedVerifications.map { it.getAttachedMessageIds(0) }) + assertEquals(messagesInCorrectOrder.map { it.metadata.id }.filter { it.sequence > 0 }, passedVerifications.map { it.getAttachedMessageIds(0) }) }, { assertCheckSequenceStatus(EventStatus.FAILED, eventsList) }) @@ -196,6 +201,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @MethodSource("checkOrderToSwitch") fun `check sequence should drop a message filter after match by key fields`(checkOrder: Boolean) { val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("AA").build(), + "B" to Value.newBuilder().setSimpleValue("BB").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -231,7 +242,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, checkOrder, filtersParam = messageFilters).begin() + sequenceCheckRuleTask(parentEventID, messageStream, checkOrder, filtersParam = messageFilters).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -255,6 +266,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @Test fun `check sequence of messages with the same value of key field`() { val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("AA").build(), + "B" to Value.newBuilder().setSimpleValue("BB").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -297,7 +314,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true, filtersParam = messageFilters).begin() + sequenceCheckRuleTask(parentEventID, messageStream, true, filtersParam = messageFilters).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -321,6 +338,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { fun `check sequence of messages with the same value of key field and message timeout`() { val checkpointTimestamp = Instant.now() val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 100)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 100)) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -369,7 +392,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { true, filtersParam = messageFilters, taskTimeout = TaskTimeout(5000L, 500L) - ).begin(createCheckpoint(checkpointTimestamp)) + ).begin(createCheckpoint(checkpointTimestamp, 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -393,6 +416,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { fun `check sequence of messages with the same value of key field and expired message timeout`() { val checkpointTimestamp = Instant.now() val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 100)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 500)) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -428,9 +457,69 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { RootMessageFilter.newBuilder(messageFilter).build() ) - val messages = Observable.fromIterable(messagesWithKeyFields) + val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesWithKeyFields) + val parentEventID = createEvent(EventUtils.generateUUID()) - val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) + sequenceCheckRuleTask( + parentEventID, + messageStream, + true, + filtersParam = messageFilters, + taskTimeout = TaskTimeout(5000L, 500L) + ).begin(createCheckpoint(checkpointTimestamp, 0)) + + val batchRequest = awaitEventBatchRequest(1000L, 6) + val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = assertNotNull(eventsList.find { it.parentId == parentEventID }) + assertEquals(1, rootEvent.attachedMessageIdsCount) + assertEquals(listOf(1L), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val checkedMessages = assertNotNull(eventsList.find { it.type == CHECK_MESSAGES_TYPE }, "Cannot find checkMessages event") + val verifications = eventsList.filter { it.parentId == checkedMessages.id } + assertEquals(1, verifications.size, "Unexpected verifications count: $verifications") + assertTrue("Some verifications are not success: $verifications") { verifications.all { it.status == EventStatus.SUCCESS } } + assertEquals(listOf(1L), verifications.flatMap { verification -> verification.attachedMessageIdsList.map { it.sequence } }) + }, { + assertCheckSequenceStatus(EventStatus.SUCCESS, eventsList) // because all key fields are in a correct order + }) + } + + @Test + fun `check sequence of messages with message timeout and missed sequence`() { + val checkpointTimestamp = Instant.now() + val messagesWithKeyFields: List = listOf( + constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 500)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(2, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 600)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build() + ) + + val messageFilter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMessageFilter( + MessageFilter.newBuilder() + .putAllFields( + mapOf( + "A" to ValueFilter.newBuilder().setKey(true).setSimpleFilter("42").build(), + "B" to ValueFilter.newBuilder().setSimpleFilter("AAA").build() + ) + ) + ).build() + val messageFilters: List = listOf( + RootMessageFilter.newBuilder(messageFilter).build() + ) + + val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesWithKeyFields) val parentEventID = createEvent(EventUtils.generateUUID()) sequenceCheckRuleTask( @@ -439,7 +528,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { true, filtersParam = messageFilters, taskTimeout = TaskTimeout(5000L, 500L) - ).begin(createCheckpoint(checkpointTimestamp)) + ).begin(createCheckpoint(checkpointTimestamp, 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -456,12 +545,20 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { assertEquals(listOf(1L), verifications.flatMap { verification -> verification.attachedMessageIdsList.map { it.sequence } }) }, { assertCheckSequenceStatus(EventStatus.SUCCESS, eventsList) // because all key fields are in a correct order + }, { + assertEquals(EventStatus.FAILED, eventsList.last().status) // check event with missed start sequence }) } @Test fun `check ordering is not failed in case key fields are matches the order but the rest are not`() { val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("AA").build(), + "B" to Value.newBuilder().setSimpleValue("BB").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -487,7 +584,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true).begin() + sequenceCheckRuleTask(parentEventID, messageStream, true).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -515,12 +612,11 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { set(indexesToSwitch.first, get(indexesToSwitch.second)) set(indexesToSwitch.second, tmp) } - val messages = Observable.fromIterable(messagesUnordered) - val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) + val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesUnordered) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, false).begin() + sequenceCheckRuleTask(parentEventID, messageStream, false).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -538,6 +634,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @Test fun `rules stops when all filters found match by key fields`() { val messagesWithKeyFields: List = listOf( + constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("AA").build(), + "B" to Value.newBuilder().setSimpleValue("BB").build() + )) + .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -574,7 +676,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, false).begin() + sequenceCheckRuleTask(parentEventID, messageStream, false).begin(createCheckpoint(sequence = 0)) val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -627,9 +729,9 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @JvmStatic fun indexesToSwitch(): Stream { return Stream.of( - arguments(0 to 1), - arguments(0 to 2), - arguments(1 to 2) + arguments(1 to 2), + arguments(1 to 3), + arguments(2 to 3) ) } @JvmStatic From 8f318cc991ad17dc738626aada447937e9af3c14 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Mon, 16 Aug 2021 12:47:51 +0400 Subject: [PATCH 07/41] [TH2-1984] Updated untrusted execution logic --- .../th2/check1/rule/AbstractCheckTask.kt | 61 +++++++--- .../th2/check1/rule/check/CheckRuleTask.kt | 2 + .../rule/sequence/SequenceCheckRuleTask.kt | 40 +------ .../th2/check1/rule/AbstractCheckTaskTest.kt | 8 ++ .../com/exactpro/th2/check1/rule/TestChain.kt | 110 ++++++++++++++++-- .../check1/rule/check/TestCheckRuleTask.kt | 80 ++++++++++++- .../rule/nomessage/TestNoMessageCheckTask.kt | 11 +- 7 files changed, 239 insertions(+), 73 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 9fd12448..3a4129ca 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -123,10 +123,8 @@ abstract class AbstractCheckTask( private var checkpointTimeout: Timestamp? = null private var lastMessageTimestamp: Timestamp? = null private var untrusted: Boolean = false - protected var hasMessagesInTimeoutInterval: Boolean = false - private set - protected var bufferContainsStartMessage: Boolean = false - private set + private var hasMessagesInTimeoutInterval: Boolean = false + private var bufferContainsStartMessage: Boolean = false override fun onStart() { super.onStart() @@ -251,12 +249,8 @@ abstract class AbstractCheckTask( LOGGER.info("Check begin for session alias '{}' with sequence '{}' and task timeout '{}'", sessionKey, sequence, taskTimeout) this.lastSequence = sequence this.executorService = executorService - if (untrusted) { - doOnUntrustedExecution().also { - this.untrusted = untrusted - taskState.set(State.ERROR) - taskFinished() - } + if (untrusted && !isIgnoreUntrustedFlag()) { + doOnUntrustedExecution() return } this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) @@ -363,8 +357,8 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - if (!untrusted) { - completeEvent(prevState) + if (!untrusted || isIgnoreUntrustedFlag()) { + doOnCompleteEvent(prevState) } _endTime = Instant.now() @@ -388,6 +382,34 @@ abstract class AbstractCheckTask( } } + private fun doOnCompleteEvent(previousState: State) { + completeEvent(previousState) + + if (!bufferContainsStartMessage) { + if (hasMessagesInTimeoutInterval) { + fillEmptyStarMessageEvent() + } else { + fillMissedStartMessageAndMessagesInIntervalEvent() + } + } + } + + private fun fillMissedStartMessageAndMessagesInIntervalEvent() { + rootEvent.addSubEvent( + Event.start() + .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval") + .status(FAILED) + ) + } + + private fun fillEmptyStarMessageEvent() { + rootEvent.addSubEvent( + Event.start() + .name("Buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' doesn't contain starting message, but contains several messages in the requested check interval") + .status(FAILED) + ) + } + protected fun matchFilter( messageContainer: MessageContainer, messageFilter: SailfishFilter, @@ -430,10 +452,21 @@ abstract class AbstractCheckTask( } /** - * This method is called before the completion of the task if the untrusted flag is set. + * This method is called before the completion of the task if the untrusted flag is set or [isIgnoreUntrustedFlag] override and returns true. * If the task has the untrusted flag, then [onComplete] will not be called. */ - protected open fun doOnUntrustedExecution() {} + protected open fun doOnUntrustedExecution() { + rootEvent.addSubEvent( + Event.start() + .name("The current check is untrusted because the start point of the check interval has been selected approximately") + .status(FAILED) + ) + this.untrusted = true + taskState.set(State.ERROR) + taskFinished() + } + + protected open fun isIgnoreUntrustedFlag(): Boolean = false companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index 36cd2050..207eaf68 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -93,4 +93,6 @@ class CheckRuleTask( .type("Check failed") .status(FAILED) } + + override fun isIgnoreUntrustedFlag(): Boolean = true } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 3d74091d..ab890fab 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -185,18 +185,9 @@ class SequenceCheckRuleTask( override fun completeEvent(taskState: State) { preFilterEvent.name("Pre-filtering (filtered ${preFilteringResults.size} / processed $handledMessageCounter) messages") - fillEvents { - fillSequenceEvent() - fillCheckMessagesEvent() - } - } - override fun doOnUntrustedExecution() { - rootEvent.addSubEvent( - Event.start() - .name("The current check is untrusted because the start point of the check interval has been selected approximately") - .status(FAILED) - ) + fillSequenceEvent() + fillCheckMessagesEvent() } /** @@ -248,33 +239,6 @@ class SequenceCheckRuleTask( .bodyData(sequenceTable.build()) } - private fun fillMissedStartMessageAndMessagesInIntervalEvent() { - rootEvent.addSubEvent( - Event.start() - .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval") - .status(FAILED) - ) - } - - private fun fillEmptyStarMessageEvent() { - rootEvent.addSubEvent( - Event.start() - .name("Buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' doesn't contain starting message, but contains several messages in the requested check interval") - .status(FAILED) - ) - } - - private fun fillEvents(commonEvents: () -> Unit) { - if (!bufferContainsStartMessage && !hasMessagesInTimeoutInterval) { - fillMissedStartMessageAndMessagesInIntervalEvent() - } else if (!bufferContainsStartMessage) { - commonEvents() - fillEmptyStarMessageEvent() - } else { - commonEvents() - } - } - companion object { const val PRE_FILTER_MESSAGE_NAME = "PreFilter" const val CHECK_MESSAGES_TYPE = "checkMessages" diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index ebe5c56c..2acb6a84 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -14,11 +14,14 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.Direction.FIRST import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.FilterOperation import com.exactpro.th2.common.grpc.Message +import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.message.toTimestamp import com.exactpro.th2.common.schema.message.MessageRouter import com.google.protobuf.Timestamp @@ -87,6 +90,11 @@ abstract class AbstractCheckTaskTest { ) }.build() + protected fun createPreFilter(fieldName: String, value: String, operation: FilterOperation): PreFilter = + PreFilter.newBuilder() + .putFields(fieldName, ValueFilter.newBuilder().setSimpleFilter(value).setKey(true).setOperation(operation).build()) + .build() + companion object { const val MESSAGE_TYPE = "TestMsg" const val SESSION_ALIAS = "test_session" diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index 4aba748c..d88c6ec6 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -17,6 +17,7 @@ import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.check.CheckRuleTask +import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask.Companion.CHECK_MESSAGES_TYPE import com.exactpro.th2.common.grpc.Direction.FIRST @@ -47,9 +48,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `simple rules - two succeed`() { - val streams = createStreams(messages = (1..3).map(::createMessage)) + val streams = createStreams(messages = (0..3).map(::createMessage)) - val task = checkRuleTask(1, eventID, streams).also { it.begin() } + val task = checkRuleTask(1, eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } var eventList = awaitEventBatchAndGetEvents(2, 2) checkSimpleVerifySuccess(eventList, 1) @@ -60,9 +61,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `simple rules - failed, succeed`() { - val streams = createStreams(messages = (1..3).map(::createMessage)) + val streams = createStreams(messages = (0..3).map(::createMessage)) - val task = checkRuleTask(4, eventID, streams).also { it.begin() } + val task = checkRuleTask(4, eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } var eventList = awaitEventBatchAndGetEvents(2, 2) checkSimpleVerifyFailure(eventList) @@ -116,7 +117,7 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `make long chain before begin`() { - val streams = createStreams(messages = (1..4).map(::createMessage)) + val streams = createStreams(messages = (0..4).map(::createMessage)) val task = checkRuleTask(1, eventID, streams).also { one -> one.subscribeNextTask(checkRuleTask(2, eventID, streams).also { two -> @@ -126,7 +127,7 @@ class TestChain: AbstractCheckTaskTest() { }) } - task.begin() + task.begin(createCheckpoint(sequence = 0)) val eventList = awaitEventBatchRequest(1000L, 4 * 2).flatMap(EventBatch::getEventsList) assertEquals(4 * 3, eventList.size) @@ -153,7 +154,7 @@ class TestChain: AbstractCheckTaskTest() { streams, taskTimeout = TaskTimeout(2000L, 500) ).also { it.begin(createCheckpoint(checkpointTimestamp, 0)) } - var eventsList = awaitEventBatchAndGetEvents(2, 2) + var eventsList = awaitEventBatchAndGetEvents(4, 4) assertAll({ val rootEvent = eventsList.first() assertEquals(FAILED, rootEvent.status, "Event status should be failed") @@ -166,10 +167,80 @@ class TestChain: AbstractCheckTaskTest() { streams, taskTimeout = TaskTimeout(2000L, 1500L) ).also { task.subscribeNextTask(it) } + eventsList = awaitEventBatchAndGetEvents(6, 2) + assertEquals("The current check is untrusted because the start point of the check interval has been selected approximately", eventsList.last().name) + } + + @Test + fun `no messages sequence rules - untrusted execution`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(messages = (1..5L).map { + constructMessage(it, timestamp = getMessageTimestamp(checkpointTimestamp, it * 1000)) + .putAllFields( + mapOf( + KEY_FIELD to "$KEY_FIELD$it".toValue(), + NOT_KEY_FIELD to "$NOT_KEY_FIELD$it".toValue() + ) + ).build() + }) + + val task = noMessageCheckTask( + eventID, + streams, + taskTimeout = TaskTimeout(2000L, 500), + preFilterParam = createPreFilter("E", "5", FilterOperation.EQUAL) + ).also { it.begin(createCheckpoint(checkpointTimestamp, 0)) } + var eventsList = awaitEventBatchAndGetEvents(2, 2) + assertAll({ + val rootEvent = eventsList.first() + assertEquals(FAILED, rootEvent.status, "Event status should be failed") + assertTrue(rootEvent.attachedMessageIdsCount == 1) + }) + + noMessageCheckTask( + eventID, + streams, + taskTimeout = TaskTimeout(2000L, 1500L), + preFilterParam = createPreFilter("E", "5", FilterOperation.EQUAL) + ).also { task.subscribeNextTask(it) } eventsList = awaitEventBatchAndGetEvents(4, 4) assertEquals("The current check is untrusted because the start point of the check interval has been selected approximately", eventsList.last().name) } + @Test + fun `simple rules - ignored untrusted execution`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(messages = (1..5L).map { + constructMessage(it, timestamp = getMessageTimestamp(checkpointTimestamp, it * 1000)) + .putAllFields( + mapOf( + KEY_FIELD to "$KEY_FIELD$it".toValue(), + NOT_KEY_FIELD to "$NOT_KEY_FIELD$it".toValue() + ) + ).build() + }) + + val task = checkRuleTask( + 1, eventID, streams, taskTimeout = TaskTimeout(2000L, 500) + ).also { it.begin(createCheckpoint(checkpointTimestamp, 0)) } + var eventsList = awaitEventBatchAndGetEvents(2, 2) + assertAll({ + val rootEvent = eventsList.first() + assertEquals(FAILED, rootEvent.status, "Event status should be failed") + assertTrue(rootEvent.attachedMessageIdsCount == 1) + }) + + checkRuleTask(3, eventID, streams).also { task.subscribeNextTask(it) } + eventsList = awaitEventBatchAndGetEvents(4, 2) + assertAll({ + val rootEvent = eventsList.first() + assertEquals(FAILED, rootEvent.status) + assertEquals(3, rootEvent.attachedMessageIdsCount) + assertEquals(1, eventsList[2].attachedMessageIdsCount) + assertEquals(FAILED, eventsList.last().status) + }) + } + private fun awaitEventBatchAndGetEvents(times: Int, last: Int): List = awaitEventBatchRequest(1000L, times).drop(times - last).flatMap(EventBatch::getEventsList) @@ -224,12 +295,13 @@ class TestChain: AbstractCheckTaskTest() { sequence: Int, parentEventID: EventID, messageStream: Observable, - maxEventBatchContentSize: Int = 1024 * 1024 + maxEventBatchContentSize: Int = 1024 * 1024, + taskTimeout: TaskTimeout = TaskTimeout(1000L) ) = CheckRuleTask( SESSION_ALIAS, Instant.now(), SessionKey(SESSION_ALIAS, FIRST), - TaskTimeout(1000L), + taskTimeout, maxEventBatchContentSize, createMessageFilter(sequence), parentEventID, @@ -237,6 +309,26 @@ class TestChain: AbstractCheckTaskTest() { clientStub ) + private fun noMessageCheckTask( + parentEventID: EventID, + messageStream: Observable, + preFilterParam: PreFilter, + taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L), + maxEventBatchContentSize: Int = 1024 * 1024 + ): NoMessageCheckTask { + return NoMessageCheckTask( + description = "Test", + startTime = Instant.now(), + sessionKey = SessionKey(SESSION_ALIAS, FIRST), + taskTimeout = taskTimeout, + maxEventBatchContentSize = maxEventBatchContentSize, + protoPreFilter = preFilterParam, + parentEventID = parentEventID, + messageStream = messageStream, + eventBatchRouter = clientStub + ) + } + private fun createMessage(sequence: Int) = constructMessage(sequence.toLong()) .putAllFields(mapOf( KEY_FIELD to "$KEY_FIELD$sequence".toValue(), diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index 393d15e8..e6dbdb59 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -63,6 +63,14 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(0L)) + .build()) + .build(), + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -74,7 +82,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() val task = checkTask(filter, eventID, streams) - task.begin() + task.begin(createCheckpoint(sequence = 0)) val eventBatches = awaitEventBatchRequest(1000L, 2) val eventList = eventBatches.flatMap(EventBatch::getEventsList) @@ -115,6 +123,14 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(0L)) + .build()) + .build(), + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -126,7 +142,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL))) .build() val task = checkTask(filter, eventID, streams, 200) - task.begin() + task.begin(createCheckpoint(sequence = 0L)) val eventBatches = awaitEventBatchRequest(1000L, 3) val eventList = eventBatches.flatMap(EventBatch::getEventsList) @@ -141,6 +157,14 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(0L)) + .build()) + .build(), + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -152,7 +176,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL))) .build() val task = checkTask(filter, eventID, streams) - task.begin() + task.begin(createCheckpoint(sequence = 0L)) val eventBatch = awaitEventBatchRequest(1000L, 2) val eventList = eventBatch.flatMap(EventBatch::getEventsList) @@ -198,6 +222,14 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { fun `success verification with message timeout`() { val checkpointTimestamp = Instant.now() val streams = createStreams(SESSION_ALIAS, Direction.FIRST, listOf( + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setTimestamp(getMessageTimestamp(checkpointTimestamp, 500)) + .setId(MessageID.newBuilder().setSequence(0L)) + .build()) + .build(), message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") @@ -215,7 +247,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(1000, 500)) - task.begin(createCheckpoint(checkpointTimestamp)) + task.begin(createCheckpoint(checkpointTimestamp, 0)) val eventBatches = awaitEventBatchRequest(1000L, 2) val eventList = eventBatches.flatMap(EventBatch::getEventsList) @@ -223,10 +255,48 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { assertEquals(4, eventList.filter { it.status == SUCCESS }.size) } + @Test + fun `success verification, but failed event because missed start message`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams(SESSION_ALIAS, Direction.FIRST, listOf( + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setTimestamp(getMessageTimestamp(checkpointTimestamp, 500)) + .setId(MessageID.newBuilder().setSequence(1L)) + .build()) + .build() + )) + + val eventID = createEvent("root") + val filter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMetadataFilter(MetadataFilter.newBuilder() + .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) + .build() + val task = checkTask(filter, eventID, streams) + task.begin() + + val eventBatches = awaitEventBatchRequest(1000L, 2) + val eventList = eventBatches.flatMap(EventBatch::getEventsList) + assertEquals(5, eventList.size) + assertEquals(2, eventList.filter { it.status == SUCCESS && it.type == "Verification" }.size) + assertEquals(FAILED, eventList.last().status) + } + @Test fun `failed verification with expired message timeout`() { val checkpointTimestamp = Instant.now() val streams = createStreams(SESSION_ALIAS, Direction.FIRST, listOf( + message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .setTimestamp(getMessageTimestamp(checkpointTimestamp, 600)) + .setId(MessageID.newBuilder().setSequence(0L)) + .build()) + .build(), message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") @@ -244,7 +314,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() val task = checkTask(filter, eventID, streams, taskTimeout = TaskTimeout(1000, 500)) - task.begin(createCheckpoint(checkpointTimestamp)) + task.begin(createCheckpoint(checkpointTimestamp, 0)) val eventBatches = awaitEventBatchRequest(1000L, 2) val eventList = eventBatches.flatMap(EventBatch::getEventsList) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index a3f2edc4..e48ccf31 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -42,6 +42,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val messageTimeout = 1500L val streams = createStreams( messages = createMessages( + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), @@ -59,7 +60,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { createPreFilter("E", "5", FilterOperation.EQUAL), TaskTimeout(5000, messageTimeout) ) - task.begin(createCheckpoint(checkpointTimestamp)) + task.begin(createCheckpoint(checkpointTimestamp, 1)) val eventBatch = awaitEventBatchRequest(1000L, 2) val eventsList = eventBatch.flatMap(EventBatch::getEventsList) @@ -78,6 +79,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val messageTimeout = 1500L val streams = createStreams( messages = createMessages( + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 50)), MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 700)), @@ -94,7 +96,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { createPreFilter("A", "1", FilterOperation.EQUAL), TaskTimeout(5000, messageTimeout) ) - task.begin(createCheckpoint(checkpointTimestamp)) + task.begin(createCheckpoint(checkpointTimestamp, 1)) val eventBatch = awaitEventBatchRequest(1000L, 2) val eventsList = eventBatch.flatMap(EventBatch::getEventsList) @@ -158,11 +160,6 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { } } - private fun createPreFilter(fieldName: String, value: String, operation: FilterOperation): PreFilter = - PreFilter.newBuilder() - .putFields(fieldName, ValueFilter.newBuilder().setSimpleFilter(value).setKey(true).setOperation(operation).build()) - .build() - private fun noMessageCheckTask( parentEventID: EventID, messageStream: Observable, From 673e3ee0b09a83fe84a9421746dcd6b7be940664 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Mon, 16 Aug 2021 19:15:48 +0400 Subject: [PATCH 08/41] [TH2-1984] Updated untrusted execution logic --- .../exactpro/th2/check1/CollectorService.kt | 17 +++++- .../th2/check1/rule/AbstractCheckTask.kt | 52 +++++++------------ .../exactpro/th2/check1/utils/TimeUtils.kt | 13 +---- .../com/exactpro/th2/check1/rule/TestChain.kt | 33 ++++++------ .../check1/rule/check/TestCheckRuleTask.kt | 32 ++---------- .../rule/sequence/TestSequenceCheckTask.kt | 52 ++++--------------- 6 files changed, 69 insertions(+), 130 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 231b63d9..354b9bdf 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -16,14 +16,27 @@ import com.exactpro.th2.check1.configuration.Check1Configuration import com.exactpro.th2.check1.entities.Checkpoint import com.exactpro.th2.check1.entities.CheckpointData import com.exactpro.th2.check1.entities.TaskTimeout -import com.exactpro.th2.check1.grpc.* +import com.exactpro.th2.check1.grpc.ChainID +import com.exactpro.th2.check1.grpc.CheckRuleRequest +import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest +import com.exactpro.th2.check1.grpc.CheckpointRequestOrBuilder +import com.exactpro.th2.check1.grpc.NoMessageCheckRequest import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.check.CheckRuleTask import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.EventUtils -import com.exactpro.th2.common.grpc.* +import com.exactpro.th2.common.grpc.ComparisonSettings +import com.exactpro.th2.common.grpc.ConnectionID +import com.exactpro.th2.common.grpc.Direction +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.MessageBatch +import com.exactpro.th2.common.grpc.MessageFilter +import com.exactpro.th2.common.grpc.MessageID +import com.exactpro.th2.common.grpc.RootComparisonSettings +import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.schema.message.MessageListener import com.exactpro.th2.common.schema.message.MessageRouter import com.exactpro.th2.common.schema.message.SubscriberMonitor diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 3a4129ca..c2527426 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -92,7 +92,7 @@ abstract class AbstractCheckTask( private val hasNextTask = AtomicBoolean(false) private val taskState = AtomicReference(State.CREATED) @Volatile - private var executionState = State.STREAM_COMPLETED + private var streamCompletedState = State.STREAM_COMPLETED /** * Used for observe messages in one thread. @@ -249,10 +249,7 @@ abstract class AbstractCheckTask( LOGGER.info("Check begin for session alias '{}' with sequence '{}' and task timeout '{}'", sessionKey, sequence, taskTimeout) this.lastSequence = sequence this.executorService = executorService - if (untrusted && !isIgnoreUntrustedFlag()) { - doOnUntrustedExecution() - return - } + this.untrusted = untrusted this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) val scheduler = Schedulers.from(executorService) @@ -308,8 +305,7 @@ abstract class AbstractCheckTask( .toProto(parentEventID)) .build()) } finally { - val untrustedForFutureTask = !bufferContainsStartMessage && !hasMessagesInTimeoutInterval - sequenceSubject.onSuccess(Legacy(executorService, SequenceData(lastSequence, lastMessageTimestamp, untrustedForFutureTask))) + sequenceSubject.onSuccess(Legacy(executorService, SequenceData(lastSequence, lastMessageTimestamp, !hasMessagesInTimeoutInterval))) } } @@ -342,7 +338,7 @@ abstract class AbstractCheckTask( override fun onComplete() { super.onComplete() - end(executionState, "Message stream is completed") + end(streamCompletedState, "Message stream is completed") } /** @@ -357,9 +353,8 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - if (!untrusted || isIgnoreUntrustedFlag()) { - doOnCompleteEvent(prevState) - } + completeEvent(prevState) + doAfterCompleteEvent() _endTime = Instant.now() val batches = rootEvent.disperseToBatches(maxEventBatchContentSize, parentEventID) @@ -382,10 +377,10 @@ abstract class AbstractCheckTask( } } - private fun doOnCompleteEvent(previousState: State) { - completeEvent(previousState) - - if (!bufferContainsStartMessage) { + private fun doAfterCompleteEvent() { + if (untrusted && !isIgnoreUntrustedFlag()) { + fillUntrustedExecutionEvent() + } else if (!bufferContainsStartMessage) { if (hasMessagesInTimeoutInterval) { fillEmptyStarMessageEvent() } else { @@ -394,6 +389,14 @@ abstract class AbstractCheckTask( } } + private fun fillUntrustedExecutionEvent() { + rootEvent.addSubEvent( + Event.start() + .name("The current check is untrusted because the start point of the check interval has been selected approximately") + .status(FAILED) + ) + } + private fun fillMissedStartMessageAndMessagesInIntervalEvent() { rootEvent.addSubEvent( Event.start() @@ -451,21 +454,6 @@ abstract class AbstractCheckTask( } } - /** - * This method is called before the completion of the task if the untrusted flag is set or [isIgnoreUntrustedFlag] override and returns true. - * If the task has the untrusted flag, then [onComplete] will not be called. - */ - protected open fun doOnUntrustedExecution() { - rootEvent.addSubEvent( - Event.start() - .name("The current check is untrusted because the start point of the check interval has been selected approximately") - .status(FAILED) - ) - this.untrusted = true - taskState.set(State.ERROR) - taskFinished() - } - protected open fun isIgnoreUntrustedFlag(): Boolean = false companion object { @@ -563,7 +551,7 @@ abstract class AbstractCheckTask( filter { streamContainer -> streamContainer.sessionKey == sessionKey } .flatMap(StreamContainer::bufferedMessages) .filter { message -> - if (message.metadata.id.sequence == sequence) { + if (message.metadata.id.sequence == sequence || sequence == DEFAULT_SEQUENCE) { bufferContainsStartMessage = true } message.metadata.id.sequence > sequence @@ -605,7 +593,7 @@ abstract class AbstractCheckTask( checkOnMessageTimeout(it.metadata.timestamp).also { continueObservation -> hasMessagesInTimeoutInterval = hasMessagesInTimeoutInterval or continueObservation if (!continueObservation) { - executionState = State.MESSAGE_TIMEOUT + streamCompletedState = State.MESSAGE_TIMEOUT } } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt index 09523a24..95085a9b 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt @@ -15,17 +15,8 @@ package com.exactpro.th2.check1.utils import com.google.protobuf.Timestamp -operator fun Timestamp.compareTo(other: Timestamp): Int { - if (this.seconds < other.seconds) - return -1 - if (this.seconds > other.seconds) - return 1 - if (this.nanos < other.nanos) - return -1 - if (this.nanos > other.nanos) - return 1 - return 0 -} +private val timestampComparator = Comparator.comparingLong(Timestamp::getSeconds).thenComparingInt(Timestamp::getNanos) +operator fun Timestamp.compareTo(other: Timestamp): Int = timestampComparator.compare(this, other) fun Timestamp.isBefore(other: Timestamp): Boolean = this < other fun Timestamp.isAfter(other: Timestamp): Boolean = this > other \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index d88c6ec6..c8d11ba4 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -48,9 +48,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `simple rules - two succeed`() { - val streams = createStreams(messages = (0..3).map(::createMessage)) + val streams = createStreams(messages = (1..3).map(::createMessage)) - val task = checkRuleTask(1, eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } + val task = checkRuleTask(1, eventID, streams).also { it.begin() } var eventList = awaitEventBatchAndGetEvents(2, 2) checkSimpleVerifySuccess(eventList, 1) @@ -61,9 +61,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `simple rules - failed, succeed`() { - val streams = createStreams(messages = (0..3).map(::createMessage)) + val streams = createStreams(messages = (1..3).map(::createMessage)) - val task = checkRuleTask(4, eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } + val task = checkRuleTask(4, eventID, streams).also { it.begin() } var eventList = awaitEventBatchAndGetEvents(2, 2) checkSimpleVerifyFailure(eventList) @@ -74,9 +74,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - two succeed`() { - val streams = createStreams(messages = (0..4).map(::createMessage)) + val streams = createStreams(messages = (1..4).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(1, 2), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } + val task = sequenceCheckRuleTask(listOf(1, 2), eventID, streams).also { it.begin() } var eventList = awaitEventBatchAndGetEvents(6, 6) checkSequenceVerifySuccess(eventList, listOf(1, 2)) @@ -87,9 +87,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - full failed, succeed`() { - val streams = createStreams(messages = (0..2).map(::createMessage)) + val streams = createStreams(messages = (1..2).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(3, 4), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } + val task = sequenceCheckRuleTask(listOf(3, 4), eventID, streams).also { it.begin() } var eventList = awaitEventBatchAndGetEvents(6, 6) assertEquals(8, eventList.size) assertEquals(4, eventList.filter { it.status == SUCCESS }.size) @@ -102,9 +102,9 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `sequence rules - part failed, succeed`() { - val streams = createStreams(messages = (0..3).map(::createMessage)) + val streams = createStreams(messages = (1..3).map(::createMessage)) - val task = sequenceCheckRuleTask(listOf(1, 4), eventID, streams).also { it.begin(createCheckpoint(sequence = 0)) } + val task = sequenceCheckRuleTask(listOf(1, 4), eventID, streams).also { it.begin() } var eventList = awaitEventBatchAndGetEvents(6, 6) assertEquals(9, eventList.size) assertEquals(5, eventList.filter { it.status == SUCCESS }.size) @@ -117,7 +117,7 @@ class TestChain: AbstractCheckTaskTest() { @Test fun `make long chain before begin`() { - val streams = createStreams(messages = (0..4).map(::createMessage)) + val streams = createStreams(messages = (1..4).map(::createMessage)) val task = checkRuleTask(1, eventID, streams).also { one -> one.subscribeNextTask(checkRuleTask(2, eventID, streams).also { two -> @@ -127,7 +127,7 @@ class TestChain: AbstractCheckTaskTest() { }) } - task.begin(createCheckpoint(sequence = 0)) + task.begin() val eventList = awaitEventBatchRequest(1000L, 4 * 2).flatMap(EventBatch::getEventsList) assertEquals(4 * 3, eventList.size) @@ -167,8 +167,8 @@ class TestChain: AbstractCheckTaskTest() { streams, taskTimeout = TaskTimeout(2000L, 1500L) ).also { task.subscribeNextTask(it) } - eventsList = awaitEventBatchAndGetEvents(6, 2) - assertEquals("The current check is untrusted because the start point of the check interval has been selected approximately", eventsList.last().name) + eventsList = awaitEventBatchAndGetEvents(10, 6) + assertEquals(UNTRUSTED_EXECUTION_EVENT_NAME, eventsList.last().name) } @Test @@ -203,8 +203,8 @@ class TestChain: AbstractCheckTaskTest() { taskTimeout = TaskTimeout(2000L, 1500L), preFilterParam = createPreFilter("E", "5", FilterOperation.EQUAL) ).also { task.subscribeNextTask(it) } - eventsList = awaitEventBatchAndGetEvents(4, 4) - assertEquals("The current check is untrusted because the start point of the check interval has been selected approximately", eventsList.last().name) + eventsList = awaitEventBatchAndGetEvents(6, 4) + assertEquals(UNTRUSTED_EXECUTION_EVENT_NAME, eventsList.last().name) } @Test @@ -348,5 +348,6 @@ class TestChain: AbstractCheckTaskTest() { companion object { private const val KEY_FIELD = "key" private const val NOT_KEY_FIELD = "not_key" + private const val UNTRUSTED_EXECUTION_EVENT_NAME: String = "The current check is untrusted because the start point of the check interval has been selected approximately" } } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index e6dbdb59..48ef3483 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -63,14 +63,6 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(0L)) - .build()) - .build(), - message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) - .mergeMetadata(MessageMetadata.newBuilder() - .putProperties("keyProp", "42") - .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -82,7 +74,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() val task = checkTask(filter, eventID, streams) - task.begin(createCheckpoint(sequence = 0)) + task.begin() val eventBatches = awaitEventBatchRequest(1000L, 2) val eventList = eventBatches.flatMap(EventBatch::getEventsList) @@ -123,14 +115,6 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(0L)) - .build()) - .build(), - message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) - .mergeMetadata(MessageMetadata.newBuilder() - .putProperties("keyProp", "42") - .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -142,7 +126,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL))) .build() val task = checkTask(filter, eventID, streams, 200) - task.begin(createCheckpoint(sequence = 0L)) + task.begin() val eventBatches = awaitEventBatchRequest(1000L, 3) val eventList = eventBatches.flatMap(EventBatch::getEventsList) @@ -157,14 +141,6 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .mergeMetadata(MessageMetadata.newBuilder() .putProperties("keyProp", "42") .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(0L)) - .build()) - .build(), - message(MESSAGE_TYPE, Direction.FIRST, SESSION_ALIAS) - .mergeMetadata(MessageMetadata.newBuilder() - .putProperties("keyProp", "42") - .putProperties("notKeyProp", "2") - .setId(MessageID.newBuilder().setSequence(1L)) .build()) .build() )) @@ -176,7 +152,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL))) .build() val task = checkTask(filter, eventID, streams) - task.begin(createCheckpoint(sequence = 0L)) + task.begin() val eventBatch = awaitEventBatchRequest(1000L, 2) val eventList = eventBatch.flatMap(EventBatch::getEventsList) @@ -276,7 +252,7 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { .putPropertyFilters("keyProp", "42".toSimpleFilter(FilterOperation.EQUAL, true))) .build() val task = checkTask(filter, eventID, streams) - task.begin() + task.begin(createCheckpoint(sequence = 0)) val eventBatches = awaitEventBatchRequest(1000L, 2) val eventList = eventBatches.flatMap(EventBatch::getEventsList) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index 19c05a5d..42590987 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -78,12 +78,6 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { ) private val messagesInCorrectOrder: List = listOf( - constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) - .putAllFields(mapOf( - "A" to Value.newBuilder().setSimpleValue("1").build(), - "B" to Value.newBuilder().setSimpleValue("A").build() - )) - .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -116,7 +110,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, checkOrder).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, checkOrder).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -176,7 +170,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesUnordered) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, true).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -191,7 +185,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { assertEquals(3, passedVerifications.size, "Unexpected SUCCESS verifications count: $passedVerifications") assertTrue("Some verifications have more than one message attached") { passedVerifications.all { it.attachedMessageIdsCount == 1 } } // Ids in the result of the rule are in order by filters because the rule creates events related to verifications/filters in the source order. - assertEquals(messagesInCorrectOrder.map { it.metadata.id }.filter { it.sequence > 0 }, passedVerifications.map { it.getAttachedMessageIds(0) }) + assertEquals(messagesInCorrectOrder.map { it.metadata.id }, passedVerifications.map { it.getAttachedMessageIds(0) }) }, { assertCheckSequenceStatus(EventStatus.FAILED, eventsList) }) @@ -201,12 +195,6 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @MethodSource("checkOrderToSwitch") fun `check sequence should drop a message filter after match by key fields`(checkOrder: Boolean) { val messagesWithKeyFields: List = listOf( - constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) - .putAllFields(mapOf( - "A" to Value.newBuilder().setSimpleValue("AA").build(), - "B" to Value.newBuilder().setSimpleValue("BB").build() - )) - .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -242,7 +230,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, checkOrder, filtersParam = messageFilters).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, checkOrder, filtersParam = messageFilters).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -266,12 +254,6 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @Test fun `check sequence of messages with the same value of key field`() { val messagesWithKeyFields: List = listOf( - constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) - .putAllFields(mapOf( - "A" to Value.newBuilder().setSimpleValue("AA").build(), - "B" to Value.newBuilder().setSimpleValue("BB").build() - )) - .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -314,7 +296,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true, filtersParam = messageFilters).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, true, filtersParam = messageFilters).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -553,12 +535,6 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @Test fun `check ordering is not failed in case key fields are matches the order but the rest are not`() { val messagesWithKeyFields: List = listOf( - constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) - .putAllFields(mapOf( - "A" to Value.newBuilder().setSimpleValue("AA").build(), - "B" to Value.newBuilder().setSimpleValue("BB").build() - )) - .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -584,7 +560,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, true).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, true).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -616,7 +592,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesUnordered) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, false).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, false).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -634,12 +610,6 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @Test fun `rules stops when all filters found match by key fields`() { val messagesWithKeyFields: List = listOf( - constructMessage(0, SESSION_ALIAS, MESSAGE_TYPE) - .putAllFields(mapOf( - "A" to Value.newBuilder().setSimpleValue("AA").build(), - "B" to Value.newBuilder().setSimpleValue("BB").build() - )) - .build(), constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE) .putAllFields(mapOf( "A" to Value.newBuilder().setSimpleValue("42").build(), @@ -676,7 +646,7 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { val messageStream: Observable = Observable.just(StreamContainer(SessionKey(SESSION_ALIAS, Direction.FIRST), 10, messages)) val parentEventID = createEvent(EventUtils.generateUUID()) - sequenceCheckRuleTask(parentEventID, messageStream, false).begin(createCheckpoint(sequence = 0)) + sequenceCheckRuleTask(parentEventID, messageStream, false).begin() val batchRequest = awaitEventBatchRequest(1000L, 6) val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) @@ -729,9 +699,9 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { @JvmStatic fun indexesToSwitch(): Stream { return Stream.of( - arguments(1 to 2), - arguments(1 to 3), - arguments(2 to 3) + arguments(0 to 1), + arguments(0 to 2), + arguments(1 to 2) ) } @JvmStatic From b2a90b7876d5a96a47e90eb3c901463d4ef4a983 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Wed, 18 Aug 2021 14:26:45 +0400 Subject: [PATCH 09/41] [TH2-1984] Remove isIgnoreUntrusted flag * Updated NoMessagesCheck tests * Added new test case with failed verification due to message timeout * Updated copyright --- .../th2/check1/entities/Checkpoint.kt | 2 +- .../th2/check1/entities/CheckpointData.kt | 2 +- .../th2/check1/entities/TaskTimeout.kt | 2 +- .../th2/check1/rule/AbstractCheckTask.kt | 16 +++-- .../th2/check1/rule/check/CheckRuleTask.kt | 2 - .../exactpro/th2/check1/utils/FilterUtils.kt | 2 +- .../th2/check1/utils/ProtoMessageUtils.kt | 2 +- .../exactpro/th2/check1/utils/TimeUtils.kt | 2 +- .../rule/nomessage/TestNoMessageCheckTask.kt | 37 ++++++++--- .../rule/sequence/TestSequenceCheckTask.kt | 64 +++++++++++++++++++ 10 files changed, 108 insertions(+), 23 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt index 27f3891f..30124200 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/Checkpoint.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt index fcb501b3..8e624daf 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/CheckpointData.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt index fee55d99..db942686 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index c2527426..c79ff19f 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -172,7 +172,7 @@ abstract class AbstractCheckTask( legacy.executorService } legacy.sequenceData.apply { - checkTask.begin(this.lastSequence, this.lastMessageTimestamp, executor, this.untrusted) + checkTask.begin(lastSequence, lastMessageTimestamp, executor, untrusted) } } LOGGER.info("Task {} ({}) subscribed to task {} ({})", checkTask.description, checkTask.hashCode(), description, hashCode()) @@ -251,6 +251,7 @@ abstract class AbstractCheckTask( this.executorService = executorService this.untrusted = untrusted this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) + this.bufferContainsStartMessage = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) @@ -378,11 +379,11 @@ abstract class AbstractCheckTask( } private fun doAfterCompleteEvent() { - if (untrusted && !isIgnoreUntrustedFlag()) { + if (untrusted) { fillUntrustedExecutionEvent() } else if (!bufferContainsStartMessage) { if (hasMessagesInTimeoutInterval) { - fillEmptyStarMessageEvent() + fillEmptyStartMessageEvent() } else { fillMissedStartMessageAndMessagesInIntervalEvent() } @@ -394,6 +395,7 @@ abstract class AbstractCheckTask( Event.start() .name("The current check is untrusted because the start point of the check interval has been selected approximately") .status(FAILED) + .type("untrustedExecution") ) } @@ -402,14 +404,16 @@ abstract class AbstractCheckTask( Event.start() .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval") .status(FAILED) + .type("missedMessagesInInterval") ) } - private fun fillEmptyStarMessageEvent() { + private fun fillEmptyStartMessageEvent() { rootEvent.addSubEvent( Event.start() .name("Buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' doesn't contain starting message, but contains several messages in the requested check interval") .status(FAILED) + .type("missedStartMessage") ) } @@ -454,8 +458,6 @@ abstract class AbstractCheckTask( } } - protected open fun isIgnoreUntrustedFlag(): Boolean = false - companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE private val RESPONSE_EXECUTOR = ForkJoinPool.commonPool() @@ -551,7 +553,7 @@ abstract class AbstractCheckTask( filter { streamContainer -> streamContainer.sessionKey == sessionKey } .flatMap(StreamContainer::bufferedMessages) .filter { message -> - if (message.metadata.id.sequence == sequence || sequence == DEFAULT_SEQUENCE) { + if (message.metadata.id.sequence == sequence) { bufferContainsStartMessage = true } message.metadata.id.sequence > sequence diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index 207eaf68..36cd2050 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -93,6 +93,4 @@ class CheckRuleTask( .type("Check failed") .status(FAILED) } - - override fun isIgnoreUntrustedFlag(): Boolean = true } diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt index 0638b864..f0cbe93e 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/FilterUtils.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt index 037f8126..672054bf 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt index 95085a9b..20c4ca5a 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/TimeUtils.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index e48ccf31..f0b06069 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 @@ -67,8 +67,15 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { assertAll({ assertTrue(eventsList.all { it.status == EventStatus.SUCCESS }, "Has messages outside the prefilter") - assertTrue(eventsList.first().attachedMessageIdsCount == 5) - assertTrue(eventsList[1].attachedMessageIdsCount == 4) + }, { + val rootEvent = eventsList.first() + assertTrue(rootEvent.attachedMessageIdsCount == 5) + assertEquals((2..6L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val prefilteredEvent = eventsList[1] + assertTrue(prefilteredEvent.attachedMessageIdsCount == 4) + assertEquals((2..5L).toList(), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) + }, { assertTrue(eventsList.last().attachedMessageIdsCount == 0) }) } @@ -105,8 +112,15 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val rootEvent = eventsList.first() assertEquals(rootEvent.status, EventStatus.FAILED, "Event status should be failed") assertTrue(rootEvent.attachedMessageIdsCount == 4) - assertTrue(eventsList[1].attachedMessageIdsCount == 2) - assertTrue(eventsList.last().attachedMessageIdsCount == 1) + assertEquals((2..5L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val prefilteredEvent = eventsList[1] + assertTrue(prefilteredEvent.attachedMessageIdsCount == 2) + assertEquals(listOf(3L, 4L), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val unexpectedMessagesEvent = eventsList.last() + assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 1) + assertEquals(listOf(2L), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) }) } @@ -139,9 +153,16 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { val rootEvent = eventsList.first() assertEquals(rootEvent.status, EventStatus.FAILED, "Root event should be failed due to timeout") assertTrue(rootEvent.attachedMessageIdsCount == 5) - assertTrue(eventsList[1].attachedMessageIdsCount == 0) - assertTrue(eventsList[1].name == "Check passed", "All messages should be ignored due to prefilter") - assertTrue(eventsList[3].attachedMessageIdsCount == 5) + assertEquals((1..5L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val prefilteredEvent = eventsList[1] + assertTrue(prefilteredEvent.attachedMessageIdsCount == 0) + assertEquals(emptyList(), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) + assertTrue(prefilteredEvent.name == "Check passed", "All messages should be ignored due to prefilter") + }, { + val unexpectedMessagesEvent = eventsList[3] + assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 5) + assertEquals((1L..5L).toList(), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) }) } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index 42590987..4996690c 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -468,6 +468,70 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { }) } + @Test + fun `check sequence of messages with the same value of key field and one missed message due to message timeout`() { + val checkpointTimestamp = Instant.now() + val messagesWithKeyFields: List = listOf( + constructMessage(1, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 500)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build(), + constructMessage(2, SESSION_ALIAS, MESSAGE_TYPE, timestamp = getMessageTimestamp(checkpointTimestamp, 600)) + .putAllFields(mapOf( + "A" to Value.newBuilder().setSimpleValue("42").build(), + "B" to Value.newBuilder().setSimpleValue("AAA").build() + )) + .build() + ) + + val messageFilter = RootMessageFilter.newBuilder() + .setMessageType(MESSAGE_TYPE) + .setMessageFilter( + MessageFilter.newBuilder() + .putAllFields( + mapOf( + "A" to ValueFilter.newBuilder().setKey(true).setSimpleFilter("42").build(), + "B" to ValueFilter.newBuilder().setSimpleFilter("AAA").build() + ) + ) + ).build() + val messageFilters: List = listOf( + RootMessageFilter.newBuilder(messageFilter).build(), + RootMessageFilter.newBuilder(messageFilter).build() + ) + + val messageStream = createStreams(SESSION_ALIAS, Direction.FIRST, messagesWithKeyFields) + val parentEventID = createEvent(EventUtils.generateUUID()) + + sequenceCheckRuleTask( + parentEventID, + messageStream, + true, + filtersParam = messageFilters, + taskTimeout = TaskTimeout(5000L, 500L) + ).begin(createCheckpoint(checkpointTimestamp, Long.MIN_VALUE)) + + val batchRequest = awaitEventBatchRequest(1000L, 6) + val eventsList: List = batchRequest.flatMap(EventBatch::getEventsList) + + assertAll({ + val rootEvent = assertNotNull(eventsList.find { it.parentId == parentEventID }) + assertEquals(2, rootEvent.attachedMessageIdsCount) + assertEquals(listOf(1L, 2L), rootEvent.attachedMessageIdsList.map { it.sequence }) + }, { + val checkedMessages = assertNotNull(eventsList.find { it.type == CHECK_MESSAGES_TYPE }, "Cannot find checkMessages event") + val verifications = eventsList.filter { it.parentId == checkedMessages.id } + assertEquals(2, verifications.size, "Unexpected verifications count: $verifications") + assertTrue("The first verification should be success") { verifications.first().status == EventStatus.SUCCESS } + assertTrue("The second verification should be failed due to message timeout") { verifications.last().status == EventStatus.FAILED } + assertEquals(listOf(1L), verifications.flatMap { verification -> verification.attachedMessageIdsList.map { it.sequence } }) + }, { + assertCheckSequenceStatus(EventStatus.FAILED, eventsList) // because the second message was skipped due to message timeout + }) + } + @Test fun `check sequence of messages with message timeout and missed sequence`() { val checkpointTimestamp = Instant.now() From 10fcbd92b1989af9ba494aaee409fb95913a8884 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 19 Aug 2021 11:25:52 +0400 Subject: [PATCH 10/41] [TH2-1984] Provided the backward compatibility with old versions of grpc-common * Provided more informative logging --- .../th2/check1/rule/AbstractCheckTask.kt | 12 +++++++----- .../th2/check1/utils/ProtoMessageUtils.kt | 16 ++++++++++++---- 2 files changed, 19 insertions(+), 9 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index c79ff19f..772fedd4 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -600,12 +600,14 @@ abstract class AbstractCheckTask( } } - private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? = - if (timestamp == null || messageTimeout == null) { - null - } else { - Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) + private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? { + if (timestamp != null && messageTimeout != null) { + return Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) + } else if (timestamp == null && messageTimeout != null){ + LOGGER.warn("Checkpoint timeout cannot be calculated because the message timeout is set, but the message timestamp is empty") } + return null + } private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) diff --git a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt index 672054bf..5eb51751 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/utils/ProtoMessageUtils.kt @@ -24,9 +24,12 @@ import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter +import mu.KotlinLogging import com.exactpro.th2.check1.entities.Checkpoint as InternalCheckpoint import com.exactpro.th2.check1.entities.CheckpointData as InternalCheckpointData +val LOGGER = KotlinLogging.logger {} + fun ProtoToIMessageConverter.fromProtoPreFilter(protoPreMessageFilter: RootMessageFilter): IMessage = fromProtoFilter(protoPreMessageFilter.messageFilter, SequenceCheckRuleTask.PRE_FILTER_MESSAGE_NAME) @@ -58,11 +61,16 @@ fun InternalCheckpoint.convert(): Checkpoint { sessionKeyToCheckpointData.forEach { (sessionKey, checkpointData) -> intermediateMap.computeIfAbsent(sessionKey.sessionAlias) { DirectionCheckpoint.newBuilder() - }.putDirectionToCheckpointData(sessionKey.direction.number, checkpointData.convert()) + }.apply { + sessionKey.direction.number.run { + putDirectionToCheckpointData(this, checkpointData.convert()) + putDirectionToSequence(this, checkpointData.sequence) + } + } } val checkpointBuilder = Checkpoint.newBuilder().setId(id) - intermediateMap.forEach { (sessionAlias, directionCheckpoint) -> + intermediateMap.forEach { (sessionAlias, directionCheckpoint) -> checkpointBuilder.putSessionAliasToDirectionCheckpoint(sessionAlias, directionCheckpoint.build()) } @@ -72,8 +80,8 @@ fun InternalCheckpoint.convert(): Checkpoint { fun Checkpoint.convert(): InternalCheckpoint { val sessionKeyToSequence: MutableMap = HashMap() sessionAliasToDirectionCheckpointMap.forEach { (sessionAlias, directionCheckpoint) -> - check(!(directionCheckpoint.run { directionToCheckpointDataCount != 0 && directionToSequenceCount != 0 })) { - "Session alias '${sessionAlias}' cannot contain both of these fields: 'direction to checkpoint data' and 'direction to sequence'. Please use 'direction to checkpoint data' instead" + if (directionCheckpoint.run { directionToCheckpointDataCount != 0 && directionToSequenceCount != 0 }) { + LOGGER.warn("Session alias '{}' contains both of these fields: 'direction to checkpoint data' and 'direction to sequence'. Please use 'direction to checkpoint data' instead", sessionAlias) } if (directionCheckpoint.directionToCheckpointDataCount == 0) { directionCheckpoint.directionToSequenceMap.forEach { (directionNumber, sequence) -> From a1ba4cb8ad755529be735793e414d1b50f6bf435 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Wed, 25 Aug 2021 11:35:58 +0400 Subject: [PATCH 11/41] [TH2-1614] Small code improvements --- .../exactpro/th2/check1/Check1Handler.java | 35 +++++++++---------- .../th2/check1/rule/AbstractCheckTask.kt | 10 +++--- 2 files changed, 23 insertions(+), 22 deletions(-) diff --git a/src/main/java/com/exactpro/th2/check1/Check1Handler.java b/src/main/java/com/exactpro/th2/check1/Check1Handler.java index 66fdf11b..ae2bbd1f 100644 --- a/src/main/java/com/exactpro/th2/check1/Check1Handler.java +++ b/src/main/java/com/exactpro/th2/check1/Check1Handler.java @@ -14,7 +14,6 @@ import static com.exactpro.th2.common.grpc.RequestStatus.Status.ERROR; import static com.exactpro.th2.common.grpc.RequestStatus.Status.SUCCESS; -import static com.google.protobuf.TextFormat.shortDebugString; import com.exactpro.th2.check1.utils.ProtoMessageUtilsKt; import com.exactpro.th2.common.message.MessageUtils; @@ -52,17 +51,17 @@ public void createCheckpoint(CheckpointRequest request, StreamObserver responseObserver) { try { if (logger.isInfoEnabled()) { - logger.info("Submit CheckRule request: " + shortDebugString(request)); + logger.info("Submit CheckRule request: " + MessageUtils.toJson(request)); } CheckRuleResponse.Builder response = CheckRuleResponse.newBuilder(); @@ -81,7 +80,7 @@ public void submitCheckRule(CheckRuleRequest request, StreamObserver responseObserver) { try { if (logger.isInfoEnabled()) { - logger.info("Submitting sequence rule for request '{}' started", MessageUtils.toJson(request)); + logger.info("Submitting no message check rule for request '{}' started", MessageUtils.toJson(request)); } NoMessageCheckResponse.Builder response = NoMessageCheckResponse.newBuilder(); @@ -166,10 +165,10 @@ public void submitNoMessageCheck(NoMessageCheckRequest request, StreamObserver Date: Thu, 26 Aug 2021 13:38:22 +0400 Subject: [PATCH 12/41] [TH2-1614] Added checkpoint verification * Updated NoMessageCheck rule logic --- .../exactpro/th2/check1/CollectorService.kt | 37 ++++++++++++++--- .../th2/check1/rule/AbstractCheckTask.kt | 22 +++++----- .../rule/nomessage/NoMessageCheckTask.kt | 40 +++++++++---------- .../rule/sequence/SequenceCheckRuleTask.kt | 3 +- .../th2/check1/rule/AbstractCheckTaskTest.kt | 7 +++- .../rule/nomessage/TestNoMessageCheckTask.kt | 40 ++++++++++++------- 6 files changed, 94 insertions(+), 55 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 354b9bdf..b92db7b0 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -89,6 +89,11 @@ class CollectorService( val parentEventID: EventID = request.parentEventId check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias + val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) + if (request.messageTimeout != 0L) { + check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } + checkCheckpoint(request.checkpoint, sessionKey) + } check(request.kindCase != CheckRuleRequest.KindCase.KIND_NOT_SET) { "Either old filter or root filter must be set" @@ -98,11 +103,10 @@ class CollectorService( } else { request.filter.toRootMessageFilter() } - val direction = directionOrDefault(request.direction) val chainID = request.getChainIdOrGenerate() - val task = CheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), + val task = CheckRuleTask(request.description, Instant.now(), sessionKey, TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, filter, parentEventID, streamObservable, eventBatchRouter) @@ -120,7 +124,11 @@ class CollectorService( val parentEventID: EventID = request.parentEventId check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias - val direction = directionOrDefault(request.direction) + val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) + if (request.messageTimeout != 0L) { + check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } + checkCheckpoint(request.checkpoint, sessionKey) + } check((request.messageFiltersList.isEmpty() && request.rootMessageFiltersList.isNotEmpty()) || (request.messageFiltersList.isNotEmpty() && request.rootMessageFiltersList.isEmpty())) { @@ -135,7 +143,7 @@ class CollectorService( request.messageFiltersList.map { it.toRootMessageFilter() } } - val task = SequenceCheckRuleTask(request.description, Instant.now(), SessionKey(sessionAlias, direction), + val task = SequenceCheckRuleTask(request.description, Instant.now(), sessionKey, TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, protoMessageFilters, request.checkOrder, parentEventID, streamObservable, eventBatchRouter) @@ -152,13 +160,17 @@ class CollectorService( val parentEventID: EventID = request.parentEventId check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias - val direction = directionOrDefault(request.direction) + val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) + if (request.messageTimeout != 0L) { + check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } + checkCheckpoint(request.checkpoint, sessionKey) + } val chainID = request.getChainIdOrGenerate() val task = NoMessageCheckTask( request.description, Instant.now(), - SessionKey(sessionAlias, direction), + sessionKey, TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, @@ -318,4 +330,17 @@ class CollectorService( .setSequence(sequence) .setDirection(direction) .build() + + private fun checkCheckpoint(checkpoint: com.exactpro.th2.common.grpc.Checkpoint, sessionKey: SessionKey) { + sessionKey.apply { + val directionCheckpoint = checkpoint.sessionAliasToDirectionCheckpointMap[sessionAlias] + check(directionCheckpoint != null) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } + val checkpointData = directionCheckpoint.directionToCheckpointDataMap[direction.number] + check(checkpointData != null) { "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" } + checkpointData.apply { + check(sequence != 0L) { "The checkpoint data has incorrect sequence number '$sequence'" } + check(this.hasTimestamp()) { "The checkpoint data doesn't contain timestamp" } + } + } + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 729c31f3..74723eb4 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -125,6 +125,7 @@ abstract class AbstractCheckTask( private var untrusted: Boolean = false private var hasMessagesInTimeoutInterval: Boolean = false private var bufferContainsStartMessage: Boolean = false + private var isDefaultSequence: Boolean = false override fun onStart() { super.onStart() @@ -251,7 +252,7 @@ abstract class AbstractCheckTask( this.executorService = executorService this.untrusted = untrusted this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) - this.bufferContainsStartMessage = sequence == DEFAULT_SEQUENCE + this.isDefaultSequence = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) @@ -381,7 +382,7 @@ abstract class AbstractCheckTask( private fun doAfterCompleteEvent() { if (untrusted) { fillUntrustedExecutionEvent() - } else if (!bufferContainsStartMessage) { + } else if (!isDefaultSequence && !bufferContainsStartMessage) { if (hasMessagesInTimeoutInterval) { fillEmptyStartMessageEvent() } else { @@ -402,7 +403,7 @@ abstract class AbstractCheckTask( private fun fillMissedStartMessageAndMessagesInIntervalEvent() { rootEvent.addSubEvent( Event.start() - .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval") + .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval with sequence '$lastSequence' and checkpoint timestamp '$checkpointTimeout'") .status(FAILED) .type("missedMessagesInInterval") ) @@ -600,16 +601,13 @@ abstract class AbstractCheckTask( } } - private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? { - if (messageTimeout != null) { - if (timestamp != null) { - return Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) - } else { - LOGGER.warn("Checkpoint timeout cannot be calculated because the message timeout is set, but the message timestamp is empty") - } + private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? = + if (timestamp != null && messageTimeout != null) { + Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) + } else { + null } - return null - } + private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index efec0f16..18653344 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -18,6 +18,7 @@ import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask +import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter import com.exactpro.th2.check1.util.VerificationUtil @@ -63,7 +64,6 @@ class NoMessageCheckTask( private lateinit var preFilterEvent: Event private lateinit var resultEvent: Event - private var preFilterMessagesCounter: Int = 0 private var extraMessagesCounter: Int = 0 init { @@ -83,31 +83,31 @@ class NoMessageCheckTask( rootEvent.addSubEvent(resultEvent) } - override fun onNext(messageContainer: MessageContainer) { - if (LOGGER.isDebugEnabled) { - LOGGER.debug("Received message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) - } + override fun Observable.taskPipeline(): Observable = + map { messageContainer -> // Compare the message with pre-filter + if (LOGGER.isDebugEnabled) { + LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) + } + val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false) + ComparisonContainer(messageContainer, protoPreMessageFilter, result) + }.filter { preFilterContainer -> // Filter check result of pre-filter + preFilterContainer.fullyMatches + }.doOnNext { preFilterContainer -> // Update pre-filter state + with(preFilterContainer) { + preFilterEvent.appendEventsWithVerification(preFilterContainer) + preFilterEvent.messageID(protoActual.metadata.id) + } + }.map(ComparisonContainer::messageContainer) - val result = matchFilter( - messageContainer, - messagePreFilter, - metadataPreFilter, - matchNames = false, - significant = false - ) + override fun onNext(messageContainer: MessageContainer) { messageContainer.protoMessage.metadata.apply { - if (FilterUtils.allMatches(result, protoPreMessageFilter) { it.fullMatch }) { - extraMessagesCounter++ - resultEvent.messageID(id) - } else { - preFilterMessagesCounter++ - preFilterEvent.messageID(id) - } + extraMessagesCounter++ + resultEvent.messageID(id) } } override fun completeEvent(taskState: State) { - preFilterEvent.name("Prefilter: $preFilterMessagesCounter messages were filtered.") + preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") if (extraMessagesCounter == 0) { resultEvent.status(Event.Status.PASSED).name("Check passed") diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index ab890fab..fec3ca77 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -39,6 +39,7 @@ import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.MessageID import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toTreeTable import com.exactpro.th2.common.schema.message.MessageRouter import com.google.protobuf.TextFormat.shortDebugString @@ -132,7 +133,7 @@ class SequenceCheckRuleTask( override fun Observable.taskPipeline(): Observable = map { messageContainer -> // Compare the message with pre-filter if (LOGGER.isDebugEnabled) { - LOGGER.debug("Pre-filtering message with id: {}", shortDebugString(messageContainer.protoMessage.metadata.id)) + LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) } val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false, significant = false) ComparisonContainer(messageContainer, protoPreMessageFilter, result) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index 2acb6a84..98ac5ef3 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -15,8 +15,10 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.grpc.PreFilter +import com.exactpro.th2.common.grpc.Checkpoint import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.Direction.FIRST +import com.exactpro.th2.common.grpc.Event import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.FilterOperation @@ -72,7 +74,7 @@ abstract class AbstractCheckTaskTest { protected fun getMessageTimestamp(start: Instant, delta: Long): Timestamp = start.plusMillis(delta).toTimestamp() - protected fun createCheckpoint(timestamp: Instant? = null, sequence: Long = -1) : com.exactpro.th2.common.grpc.Checkpoint = + protected fun createCheckpoint(timestamp: Instant? = null, sequence: Long = -1) : Checkpoint = com.exactpro.th2.common.grpc.Checkpoint.newBuilder().apply { putSessionAliasToDirectionCheckpoint( SESSION_ALIAS, @@ -95,6 +97,9 @@ abstract class AbstractCheckTaskTest { .putFields(fieldName, ValueFilter.newBuilder().setSimpleFilter(value).setKey(true).setOperation(operation).build()) .build() + protected fun List.findEventByType(eventType: String): Event? = + this.find { it.type == eventType } + companion object { const val MESSAGE_TYPE = "TestMsg" const val SESSION_ALIAS = "test_session" diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index f0b06069..55be7e53 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -25,7 +25,6 @@ import com.exactpro.th2.common.grpc.EventStatus import com.exactpro.th2.common.grpc.FilterOperation import com.exactpro.th2.common.grpc.Message import com.exactpro.th2.common.grpc.Value -import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.value.toValue import com.google.protobuf.Timestamp import io.reactivex.Observable @@ -33,6 +32,7 @@ import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertAll import java.time.Instant import kotlin.test.assertEquals +import kotlin.test.assertNotNull import kotlin.test.assertTrue class TestNoMessageCheckTask : AbstractCheckTaskTest() { @@ -72,11 +72,15 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { assertTrue(rootEvent.attachedMessageIdsCount == 5) assertEquals((2..6L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) }, { - val prefilteredEvent = eventsList[1] - assertTrue(prefilteredEvent.attachedMessageIdsCount == 4) - assertEquals((2..5L).toList(), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) + val prefilteredEvent = eventsList.findEventByType("preFiltering") + assertNotNull(prefilteredEvent, "Missed pre filtering event") + assertTrue(prefilteredEvent.attachedMessageIdsCount == 0) + assertEquals(emptyList(), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) }, { - assertTrue(eventsList.last().attachedMessageIdsCount == 0) + val unexpectedMessagesEvent = eventsList.findEventByType("noMessagesCheckResult") + assertNotNull(unexpectedMessagesEvent, "Missed resulting event") + assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 0) + assertEquals(emptyList(), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) }) } @@ -105,7 +109,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { ) task.begin(createCheckpoint(checkpointTimestamp, 1)) - val eventBatch = awaitEventBatchRequest(1000L, 2) + val eventBatch = awaitEventBatchRequest(1000L, 4) val eventsList = eventBatch.flatMap(EventBatch::getEventsList) assertAll({ @@ -114,11 +118,15 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { assertTrue(rootEvent.attachedMessageIdsCount == 4) assertEquals((2..5L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) }, { - val prefilteredEvent = eventsList[1] - assertTrue(prefilteredEvent.attachedMessageIdsCount == 2) - assertEquals(listOf(3L, 4L), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) + val prefilteredEvent = eventsList.findEventByType("preFiltering") + assertNotNull(prefilteredEvent, "Missed pre filtering event") + assertTrue(prefilteredEvent.attachedMessageIdsCount == 1) + val verificationEvents = eventsList.filter { it.type == "Verification" } + assertTrue(verificationEvents.all { it.parentId == prefilteredEvent.id }) + assertEquals(listOf(2L), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) }, { - val unexpectedMessagesEvent = eventsList.last() + val unexpectedMessagesEvent = eventsList.findEventByType("noMessagesCheckResult") + assertNotNull(unexpectedMessagesEvent, "Missed resulting event") assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 1) assertEquals(listOf(2L), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) }) @@ -155,14 +163,16 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { assertTrue(rootEvent.attachedMessageIdsCount == 5) assertEquals((1..5L).toList(), rootEvent.attachedMessageIdsList.map { it.sequence }) }, { - val prefilteredEvent = eventsList[1] + val prefilteredEvent = eventsList.findEventByType("preFiltering") + assertNotNull(prefilteredEvent, "Missed pre filtering event") assertTrue(prefilteredEvent.attachedMessageIdsCount == 0) assertEquals(emptyList(), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) - assertTrue(prefilteredEvent.name == "Check passed", "All messages should be ignored due to prefilter") }, { - val unexpectedMessagesEvent = eventsList[3] - assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 5) - assertEquals((1L..5L).toList(), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) + val unexpectedMessagesEvent = eventsList.findEventByType("noMessagesCheckResult") + assertNotNull(unexpectedMessagesEvent, "Missed resulting event") + assertTrue(unexpectedMessagesEvent.attachedMessageIdsCount == 0) + assertEquals(emptyList(), unexpectedMessagesEvent.attachedMessageIdsList.map { it.sequence }) + assertTrue(unexpectedMessagesEvent.name == "Check passed", "All messages should be ignored due to prefilter") }) } From bb2cd2c66c9337bcff58ec5e1fed09e2dace9242 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 26 Aug 2021 18:05:30 +0400 Subject: [PATCH 13/41] [TH2-1614] Improved message timeout and checkpoint checking --- .../exactpro/th2/check1/CollectorService.kt | 32 ++++++++++--------- .../rule/nomessage/TestNoMessageCheckTask.kt | 1 + 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index b92db7b0..e5d6fac1 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -50,6 +50,7 @@ import java.time.Instant import java.time.temporal.ChronoUnit import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ForkJoinPool +import com.exactpro.th2.common.grpc.Checkpoint as GrpcCheckpoint class CollectorService( private val messageRouter: MessageRouter, private val eventBatchRouter: MessageRouter, configuration: Check1Configuration @@ -90,10 +91,7 @@ class CollectorService( check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - if (request.messageTimeout != 0L) { - check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } - checkCheckpoint(request.checkpoint, sessionKey) - } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } check(request.kindCase != CheckRuleRequest.KindCase.KIND_NOT_SET) { "Either old filter or root filter must be set" @@ -125,10 +123,7 @@ class CollectorService( check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - if (request.messageTimeout != 0L) { - check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } - checkCheckpoint(request.checkpoint, sessionKey) - } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } check((request.messageFiltersList.isEmpty() && request.rootMessageFiltersList.isNotEmpty()) || (request.messageFiltersList.isNotEmpty() && request.rootMessageFiltersList.isEmpty())) { @@ -161,10 +156,8 @@ class CollectorService( check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - if (request.messageTimeout != 0L) { - check(request.hasCheckpoint()) { "Request doesn't contain a checkpoint" } - checkCheckpoint(request.checkpoint, sessionKey) - } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } + val chainID = request.getChainIdOrGenerate() val task = NoMessageCheckTask( @@ -207,7 +200,7 @@ class CollectorService( private fun AbstractCheckTask.addToChainOrBegin( value: AbstractCheckTask?, - checkpoint: com.exactpro.th2.common.grpc.Checkpoint + checkpoint: GrpcCheckpoint ): Unit = value?.subscribeNextTask(this) ?: begin(checkpoint) private fun CheckRuleRequest.getChainIdOrGenerate(): ChainID { @@ -331,8 +324,9 @@ class CollectorService( .setDirection(direction) .build() - private fun checkCheckpoint(checkpoint: com.exactpro.th2.common.grpc.Checkpoint, sessionKey: SessionKey) { - sessionKey.apply { + private fun checkCheckpoint(checkpoint: GrpcCheckpoint, sessionKey: SessionKey) { + check(checkpoint !== GrpcCheckpoint.getDefaultInstance()) { "Request doesn't contain a checkpoint" } + with(sessionKey) { val directionCheckpoint = checkpoint.sessionAliasToDirectionCheckpointMap[sessionAlias] check(directionCheckpoint != null) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } val checkpointData = directionCheckpoint.directionToCheckpointDataMap[direction.number] @@ -343,4 +337,12 @@ class CollectorService( } } } + + private fun checkMessageTimeout(messageTimeout: Long, checkpointCheckAction: () -> Unit){ + if (messageTimeout != 0L) { + checkpointCheckAction() + } else if (messageTimeout < 0) { + throw IllegalStateException("Message timeout cannot be negative") + } + } } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index 55be7e53..fb8e381c 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -122,6 +122,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { assertNotNull(prefilteredEvent, "Missed pre filtering event") assertTrue(prefilteredEvent.attachedMessageIdsCount == 1) val verificationEvents = eventsList.filter { it.type == "Verification" } + assertEquals(1, verificationEvents.size) assertTrue(verificationEvents.all { it.parentId == prefilteredEvent.id }) assertEquals(listOf(2L), prefilteredEvent.attachedMessageIdsList.map { it.sequence }) }, { From 3a00a5f0d5798107d93b068dec678a2fd2d019f8 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Fri, 27 Aug 2021 15:23:10 +0400 Subject: [PATCH 14/41] [TH2-1614] Improved message timeout and checkpoint checking --- .../com/exactpro/th2/check1/CollectorService.kt | 13 ++++++------- .../exactpro/th2/check1/rule/AbstractCheckTask.kt | 2 +- .../th2/check1/rule/nomessage/NoMessageCheckTask.kt | 7 +++---- .../th2/check1/rule/AbstractCheckTaskTest.kt | 6 +++--- 4 files changed, 13 insertions(+), 15 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index e5d6fac1..312187c9 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -331,18 +331,17 @@ class CollectorService( check(directionCheckpoint != null) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } val checkpointData = directionCheckpoint.directionToCheckpointDataMap[direction.number] check(checkpointData != null) { "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" } - checkpointData.apply { - check(sequence != 0L) { "The checkpoint data has incorrect sequence number '$sequence'" } + with(checkpointData) { + check(sequence > 0L) { "The checkpoint data has incorrect sequence number '$sequence'" } check(this.hasTimestamp()) { "The checkpoint data doesn't contain timestamp" } } } } - private fun checkMessageTimeout(messageTimeout: Long, checkpointCheckAction: () -> Unit){ - if (messageTimeout != 0L) { - checkpointCheckAction() - } else if (messageTimeout < 0) { - throw IllegalStateException("Message timeout cannot be negative") + private fun checkMessageTimeout(messageTimeout: Long, checkpointCheckAction: () -> Unit) { + when { + messageTimeout > 0 -> checkpointCheckAction() + messageTimeout < 0 -> error("Message timeout cannot be negative") } } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 74723eb4..3ef64aea 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -403,7 +403,7 @@ abstract class AbstractCheckTask( private fun fillMissedStartMessageAndMessagesInIntervalEvent() { rootEvent.addSubEvent( Event.start() - .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval with sequence '$lastSequence' and checkpoint timestamp '$checkpointTimeout'") + .name("Check cannot be executed because buffer for session alias '${sessionKey.sessionAlias}' and direction '${sessionKey.direction}' contains neither message in the requested check interval with sequence '$lastSequence' and checkpoint timestamp '${checkpointTimeout?.toJson()}'") .status(FAILED) .type("missedMessagesInInterval") ) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 18653344..69c47073 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) * 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 @@ -22,8 +22,6 @@ import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter import com.exactpro.th2.check1.util.VerificationUtil -import com.exactpro.th2.check1.utils.FilterUtils -import com.exactpro.th2.check1.utils.FilterUtils.fullMatch import com.exactpro.th2.check1.utils.fromProtoPreFilter import com.exactpro.th2.check1.utils.toRootMessageFilter import com.exactpro.th2.common.event.Event @@ -46,7 +44,7 @@ class NoMessageCheckTask( parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter - ) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( @@ -83,6 +81,7 @@ class NoMessageCheckTask( rootEvent.addSubEvent(resultEvent) } + //TODO: Need to extract logic from here and from the SequenceCheck rule to some utility class and try to optimize that logic override fun Observable.taskPipeline(): Observable = map { messageContainer -> // Compare the message with pre-filter if (LOGGER.isDebugEnabled) { diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index 98ac5ef3..3a2f4c4c 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -75,13 +75,13 @@ abstract class AbstractCheckTaskTest { start.plusMillis(delta).toTimestamp() protected fun createCheckpoint(timestamp: Instant? = null, sequence: Long = -1) : Checkpoint = - com.exactpro.th2.common.grpc.Checkpoint.newBuilder().apply { + Checkpoint.newBuilder().apply { putSessionAliasToDirectionCheckpoint( SESSION_ALIAS, - com.exactpro.th2.common.grpc.Checkpoint.DirectionCheckpoint.newBuilder().apply { + Checkpoint.DirectionCheckpoint.newBuilder().apply { putDirectionToCheckpointData( FIRST.number, - com.exactpro.th2.common.grpc.Checkpoint.CheckpointData.newBuilder().apply { + Checkpoint.CheckpointData.newBuilder().apply { this.sequence = sequence if (timestamp != null) { this.timestamp = timestamp.toTimestamp() From 2ca2c0ff956e70664bfaf1ee10600a291741c7e0 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 9 Sep 2021 16:31:26 +0400 Subject: [PATCH 15/41] [TH2-1950] Added check that checkpoint is the last received message in the buffer * Added the ability to specify only the message timeout --- .../th2/check1/rule/AbstractCheckTask.kt | 16 +++++- .../rule/nomessage/NoMessageCheckTask.kt | 10 ++-- .../rule/nomessage/TestNoMessageCheckTask.kt | 50 +++++++++++++++++-- 3 files changed, 68 insertions(+), 8 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 68d8ca85..a4598899 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -126,6 +126,8 @@ abstract class AbstractCheckTask( private var hasMessagesInTimeoutInterval: Boolean = false private var bufferContainsStartMessage: Boolean = false private var isDefaultSequence: Boolean = false + protected var isCheckpointLastReceivedMessage = bufferContainsStartMessage && !hasMessagesInTimeoutInterval + private set override fun onStart() { super.onStart() @@ -255,7 +257,7 @@ abstract class AbstractCheckTask( this.isDefaultSequence = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) - endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) + endFuture = Single.timer(taskTimeout.getOrCalculateMessageTimeout(), MILLISECONDS, Schedulers.computation()) .subscribe { _ -> end(State.TIMEOUT, "Timeout is exited") } messageStream.observeOn(scheduler) // Defined scheduler to execution in one thread to avoid race-condition. @@ -608,6 +610,18 @@ abstract class AbstractCheckTask( null } + private fun TaskTimeout.getOrCalculateMessageTimeout(): Long { + if (this.timeout > 0L) { + return timeout + } + require(messageTimeout != null && messageTimeout > 0) { + "Timeout cannot be calculated because 'timeout' and 'message timeout' is not set" + } + return messageTimeout.also { + LOGGER.info("Rule `timeout` is not specified, used `message timeout` instead") + } + } + private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 69c47073..3860f07d 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -116,9 +116,13 @@ class NoMessageCheckTask( } if (taskState == State.TIMEOUT || taskState == State.STREAM_COMPLETED) { - resultEvent.addSubEvent( - Event.start().name("Task has been completed because: ${taskState.name}").status(Event.Status.FAILED) - ) + val executionStopEvent = Event.start() + .name("Task has been completed because: ${taskState.name}") + .type("noMessageCheckExecutionStop") + if (taskState != State.TIMEOUT || !isCheckpointLastReceivedMessage) { + executionStopEvent.status(Event.Status.FAILED) + } + resultEvent.addSubEvent(executionStopEvent) } } } \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index fb8e381c..05c1f5ad 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -30,16 +30,21 @@ import com.google.protobuf.Timestamp import io.reactivex.Observable import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertAll +import org.junit.jupiter.api.assertThrows +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.MethodSource import java.time.Instant +import java.util.stream.Stream import kotlin.test.assertEquals import kotlin.test.assertNotNull import kotlin.test.assertTrue class TestNoMessageCheckTask : AbstractCheckTaskTest() { - @Test - fun `no messages outside the prefilter`() { + @ParameterizedTest(name = "TaskTimeout = {0}") + @MethodSource("taskTimeouts") + fun `no messages outside the prefilter`(taskTimeout: TaskTimeout) { val checkpointTimestamp = Instant.now() - val messageTimeout = 1500L val streams = createStreams( messages = createMessages( MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), @@ -58,7 +63,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { eventID, streams, createPreFilter("E", "5", FilterOperation.EQUAL), - TaskTimeout(5000, messageTimeout) + taskTimeout ) task.begin(createCheckpoint(checkpointTimestamp, 1)) @@ -177,6 +182,33 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { }) } + @Test + fun `rule cannot be create due to missed timeout and message timeout`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams( + messages = createMessages( + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), + MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), + MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), + // should be skipped because of message timeout + MessageData("F", "6".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) + ) + ) + + val exception = assertThrows { + noMessageCheckTask( + createEvent("root"), + streams, + createPreFilter("E", "5", FilterOperation.EQUAL), + TaskTimeout(0) + ).begin(createCheckpoint(checkpointTimestamp, 1)) + } + assertEquals("Timeout cannot be calculated because 'timeout' and 'message timeout' is not set", exception.message) + } + private fun createMessages( vararg messageData: MessageData, @@ -212,5 +244,15 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { ) } + companion object { + @JvmStatic + fun taskTimeouts(): Stream { + return Stream.of( + Arguments.arguments(TaskTimeout(5000, 1500)), // with timeout and message timeout + Arguments.arguments(TaskTimeout(0, 1500)) // with message timeout and missed timeout + ) + } + } + data class MessageData(val fieldName: String, val value: Value, val timestamp: Timestamp) } \ No newline at end of file From 736c2684c375c48158c52b3ffbf662108c8e3a32 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Mon, 13 Sep 2021 15:23:05 +0400 Subject: [PATCH 16/41] [TH2-1950] Small improvements --- .../th2/check1/entities/TaskTimeout.kt | 2 +- .../th2/check1/rule/AbstractCheckTask.kt | 35 +++++++++++++------ .../rule/nomessage/NoMessageCheckTask.kt | 2 +- .../rule/nomessage/TestNoMessageCheckTask.kt | 29 ++++++++++++++- 4 files changed, 54 insertions(+), 14 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt index db942686..04c94cfd 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/TaskTimeout.kt @@ -13,4 +13,4 @@ package com.exactpro.th2.check1.entities -data class TaskTimeout(val timeout: Long, val messageTimeout: Long? = null) \ No newline at end of file +data class TaskTimeout(val timeout: Long, val messageTimeout: Long = 0) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index a4598899..03b1c356 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -41,6 +41,7 @@ import com.exactpro.th2.common.grpc.MetadataFilter import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.message.toReadableBodyCollection import com.exactpro.th2.common.message.toJson +import com.exactpro.th2.common.message.toTimestamp import com.exactpro.th2.common.schema.message.MessageRouter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter import com.google.protobuf.TextFormat.shortDebugString @@ -126,8 +127,6 @@ abstract class AbstractCheckTask( private var hasMessagesInTimeoutInterval: Boolean = false private var bufferContainsStartMessage: Boolean = false private var isDefaultSequence: Boolean = false - protected var isCheckpointLastReceivedMessage = bufferContainsStartMessage && !hasMessagesInTimeoutInterval - private set override fun onStart() { super.onStart() @@ -257,7 +256,7 @@ abstract class AbstractCheckTask( this.isDefaultSequence = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) - endFuture = Single.timer(taskTimeout.getOrCalculateMessageTimeout(), MILLISECONDS, Schedulers.computation()) + endFuture = Single.timer(taskTimeout.getOrCalculateTimeout(checkpointTimestamp), MILLISECONDS, Schedulers.computation()) .subscribe { _ -> end(State.TIMEOUT, "Timeout is exited") } messageStream.observeOn(scheduler) // Defined scheduler to execution in one thread to avoid race-condition. @@ -351,6 +350,8 @@ abstract class AbstractCheckTask( */ protected open fun completeEvent(taskState: State) {} + protected fun isCheckpointLastReceivedMessage(): Boolean = bufferContainsStartMessage && !hasMessagesInTimeoutInterval + /** * Publishes the event to [eventBatchRouter]. */ @@ -463,6 +464,7 @@ abstract class AbstractCheckTask( companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE + const val DEFAULT_TASK_TIMEOUT = 3000L private val RESPONSE_EXECUTOR = ForkJoinPool.commonPool() } @@ -603,22 +605,33 @@ abstract class AbstractCheckTask( } } - private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long?): Timestamp? = - if (timestamp != null && messageTimeout != null) { + private fun calculateCheckpointTimeout(timestamp: Timestamp?, messageTimeout: Long): Timestamp? = + if (timestamp != null && messageTimeout > 0) { Timestamps.add(timestamp, Durations.fromMillis(messageTimeout)) } else { null } - private fun TaskTimeout.getOrCalculateMessageTimeout(): Long { - if (this.timeout > 0L) { + private fun TaskTimeout.getOrCalculateTimeout(checkpointTimestamp: Timestamp?): Long { + if (timeout > 0L) { + if (messageTimeout > 0L) { + require(timeout > messageTimeout) { "The 'timeout' should be greater than the 'message timeout'" } + } return timeout } - require(messageTimeout != null && messageTimeout > 0) { - "Timeout cannot be calculated because 'timeout' and 'message timeout' is not set" + require(messageTimeout > 0) { + "Timeout cannot be calculated because 'timeout' and 'message timeout' is not specified or negative" } - return messageTimeout.also { - LOGGER.info("Rule `timeout` is not specified, used `message timeout` instead") + checkNotNull(checkpointTimestamp) { "Timeout cannot be calculated because 'checkpoint timestamp' is not specified" } + val timestampDiff = Timestamps.between(Instant.now().toTimestamp(), checkpointTimestamp) + val durationDiff = Durations.subtract(Durations.fromMillis(timeout), timestampDiff) + val newTimeout = Durations.toMillis(durationDiff) + return if (newTimeout < 0) { + DEFAULT_TASK_TIMEOUT + } else if (newTimeout < 1000L) { + 1000L + } else { + newTimeout } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 3860f07d..c8b8f3e1 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -119,7 +119,7 @@ class NoMessageCheckTask( val executionStopEvent = Event.start() .name("Task has been completed because: ${taskState.name}") .type("noMessageCheckExecutionStop") - if (taskState != State.TIMEOUT || !isCheckpointLastReceivedMessage) { + if (taskState != State.TIMEOUT || !isCheckpointLastReceivedMessage()) { executionStopEvent.status(Event.Status.FAILED) } resultEvent.addSubEvent(executionStopEvent) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index 05c1f5ad..014b7f75 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -206,7 +206,34 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { TaskTimeout(0) ).begin(createCheckpoint(checkpointTimestamp, 1)) } - assertEquals("Timeout cannot be calculated because 'timeout' and 'message timeout' is not set", exception.message) + assertEquals("Timeout cannot be calculated because 'timeout' and 'message timeout' is not specified or negative", exception.message) + } + + @Test + fun `rule cannot be create because specified timeout less than message timeout`() { + val checkpointTimestamp = Instant.now() + val streams = createStreams( + messages = createMessages( + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), + MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), + MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), + MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), + MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), + MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), + // should be skipped because of message timeout + MessageData("F", "6".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) + ) + ) + + val exception = assertThrows { + noMessageCheckTask( + createEvent("root"), + streams, + createPreFilter("E", "5", FilterOperation.EQUAL), + TaskTimeout(1000, 5000) + ).begin(createCheckpoint(checkpointTimestamp, 1)) + } + assertEquals("The 'timeout' should be greater than the 'message timeout'", exception.message) } From 34c96056abaf506db0d63ca4c8873e5a9dfd6ade Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 23 Sep 2021 13:25:35 +0400 Subject: [PATCH 17/41] [TH2-1950] New configuration parameter 'rule-execution-timeout' --- README.md | 5 +- .../configuration/Check1Configuration.java | 7 ++ .../exactpro/th2/check1/CollectorService.kt | 49 +++++++++--- .../th2/check1/rule/AbstractCheckTask.kt | 25 +------ .../rule/nomessage/TestNoMessageCheckTask.kt | 75 +------------------ 5 files changed, 54 insertions(+), 107 deletions(-) diff --git a/README.md b/README.md index 498a5197..83d47dd4 100644 --- a/README.md +++ b/README.md @@ -68,7 +68,8 @@ This block describes the configuration for check1. "message-cache-size": 1000, "cleanup-older-than": 60, "cleanup-time-unit": "SECONDS", - "max-event-batch-content-size": "1048576" + "max-event-batch-content-size": "1048576", + "rule-execution-timeout": 5000 } ``` @@ -113,7 +114,9 @@ spec: ### 3.8.0 +#### Added: + Implemented NoMessageCheck rule task. Updated CheckRule and CheckSequence rule tasks ++ New configuration parameter `rule-execution-timeout` witch is used if the user has not specified a timeout for the rule execution ### 3.7.2 diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index f43653e1..28073d83 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -31,6 +31,9 @@ public class Check1Configuration { @JsonProperty(value="cleanup-time-unit", defaultValue = "SECONDS") private ChronoUnit cleanupTimeUnit = ChronoUnit.SECONDS; + @JsonProperty(value="rule-execution-timeout", defaultValue = "5000") + private long ruleExecutionTimeout = 5000L; + public int getMessageCacheSize() { return messageCacheSize; } @@ -46,4 +49,8 @@ public int getMaxEventBatchContentSize() { public ChronoUnit getCleanupTimeUnit() { return cleanupTimeUnit; } + + public long getRuleExecutionTimeout() { + return ruleExecutionTimeout; + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 312187c9..9e70a24b 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -70,6 +70,7 @@ class CollectorService( private val olderThanDelta = configuration.cleanupOlderThan private val olderThanTimeUnit = configuration.cleanupTimeUnit private val maxEventBatchContentSize = configuration.maxEventBatchContentSize + private val defaultRuleExecutionTimeout = configuration.ruleExecutionTimeout init { val limitSize = configuration.messageCacheSize @@ -104,9 +105,17 @@ class CollectorService( val chainID = request.getChainIdOrGenerate() - val task = CheckRuleTask(request.description, Instant.now(), sessionKey, - TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, - filter, parentEventID, streamObservable, eventBatchRouter) + val task = CheckRuleTask( + request.description, + Instant.now(), + sessionKey, + createTaskTimeout(request.timeout, request.messageTimeout), + maxEventBatchContentSize, + filter, + parentEventID, + streamObservable, + eventBatchRouter + ) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -137,10 +146,20 @@ class CollectorService( } else { request.messageFiltersList.map { it.toRootMessageFilter() } } - - val task = SequenceCheckRuleTask(request.description, Instant.now(), sessionKey, - TaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, - protoMessageFilters, request.checkOrder, parentEventID, streamObservable, eventBatchRouter) + + val task = SequenceCheckRuleTask( + request.description, + Instant.now(), + sessionKey, + createTaskTimeout(request.timeout, request.messageTimeout), + maxEventBatchContentSize, + request.preFilter, + protoMessageFilters, + request.checkOrder, + parentEventID, + streamObservable, + eventBatchRouter + ) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) @@ -164,7 +183,7 @@ class CollectorService( request.description, Instant.now(), sessionKey, - TaskTimeout(request.timeout, request.messageTimeout), + createTaskTimeout(request.timeout, request.messageTimeout), maxEventBatchContentSize, request.preFilter, parentEventID, @@ -328,9 +347,9 @@ class CollectorService( check(checkpoint !== GrpcCheckpoint.getDefaultInstance()) { "Request doesn't contain a checkpoint" } with(sessionKey) { val directionCheckpoint = checkpoint.sessionAliasToDirectionCheckpointMap[sessionAlias] - check(directionCheckpoint != null) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } + checkNotNull(directionCheckpoint) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } val checkpointData = directionCheckpoint.directionToCheckpointDataMap[direction.number] - check(checkpointData != null) { "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" } + checkNotNull(checkpointData) { "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" } with(checkpointData) { check(sequence > 0L) { "The checkpoint data has incorrect sequence number '$sequence'" } check(this.hasTimestamp()) { "The checkpoint data doesn't contain timestamp" } @@ -344,4 +363,14 @@ class CollectorService( messageTimeout < 0 -> error("Message timeout cannot be negative") } } + + private fun createTaskTimeout(timeout: Long, messageTimeout: Long): TaskTimeout { + val newRuleTimeout = if (timeout <= 0) { + logger.info("Rule execution timeout is less than or equal to zero, used default rule execution timeout '$defaultRuleExecutionTimeout'") + defaultRuleExecutionTimeout + } else { + timeout + } + return TaskTimeout(newRuleTimeout, messageTimeout) + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 03b1c356..9ac67be0 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -256,7 +256,7 @@ abstract class AbstractCheckTask( this.isDefaultSequence = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) - endFuture = Single.timer(taskTimeout.getOrCalculateTimeout(checkpointTimestamp), MILLISECONDS, Schedulers.computation()) + endFuture = Single.timer(taskTimeout.timeout, MILLISECONDS, Schedulers.computation()) .subscribe { _ -> end(State.TIMEOUT, "Timeout is exited") } messageStream.observeOn(scheduler) // Defined scheduler to execution in one thread to avoid race-condition. @@ -612,29 +612,6 @@ abstract class AbstractCheckTask( null } - private fun TaskTimeout.getOrCalculateTimeout(checkpointTimestamp: Timestamp?): Long { - if (timeout > 0L) { - if (messageTimeout > 0L) { - require(timeout > messageTimeout) { "The 'timeout' should be greater than the 'message timeout'" } - } - return timeout - } - require(messageTimeout > 0) { - "Timeout cannot be calculated because 'timeout' and 'message timeout' is not specified or negative" - } - checkNotNull(checkpointTimestamp) { "Timeout cannot be calculated because 'checkpoint timestamp' is not specified" } - val timestampDiff = Timestamps.between(Instant.now().toTimestamp(), checkpointTimestamp) - val durationDiff = Durations.subtract(Durations.fromMillis(timeout), timestampDiff) - val newTimeout = Durations.toMillis(durationDiff) - return if (newTimeout < 0) { - DEFAULT_TASK_TIMEOUT - } else if (newTimeout < 1000L) { - 1000L - } else { - newTimeout - } - } - private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index 014b7f75..e1d9ca04 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -30,20 +30,14 @@ import com.google.protobuf.Timestamp import io.reactivex.Observable import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertAll -import org.junit.jupiter.api.assertThrows -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments -import org.junit.jupiter.params.provider.MethodSource import java.time.Instant -import java.util.stream.Stream import kotlin.test.assertEquals import kotlin.test.assertNotNull import kotlin.test.assertTrue class TestNoMessageCheckTask : AbstractCheckTaskTest() { - @ParameterizedTest(name = "TaskTimeout = {0}") - @MethodSource("taskTimeouts") - fun `no messages outside the prefilter`(taskTimeout: TaskTimeout) { + @Test + fun `no messages outside the prefilter`() { val checkpointTimestamp = Instant.now() val streams = createStreams( messages = createMessages( @@ -63,7 +57,7 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { eventID, streams, createPreFilter("E", "5", FilterOperation.EQUAL), - taskTimeout + TaskTimeout(5000L, 1500L) ) task.begin(createCheckpoint(checkpointTimestamp, 1)) @@ -182,60 +176,6 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { }) } - @Test - fun `rule cannot be create due to missed timeout and message timeout`() { - val checkpointTimestamp = Instant.now() - val streams = createStreams( - messages = createMessages( - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), - MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), - MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), - MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), - // should be skipped because of message timeout - MessageData("F", "6".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) - ) - ) - - val exception = assertThrows { - noMessageCheckTask( - createEvent("root"), - streams, - createPreFilter("E", "5", FilterOperation.EQUAL), - TaskTimeout(0) - ).begin(createCheckpoint(checkpointTimestamp, 1)) - } - assertEquals("Timeout cannot be calculated because 'timeout' and 'message timeout' is not specified or negative", exception.message) - } - - @Test - fun `rule cannot be create because specified timeout less than message timeout`() { - val checkpointTimestamp = Instant.now() - val streams = createStreams( - messages = createMessages( - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 100)), - MessageData("A", "1".toValue(), getMessageTimestamp(checkpointTimestamp, 500)), - MessageData("B", "2".toValue(), getMessageTimestamp(checkpointTimestamp, 1000)), - MessageData("C", "3".toValue(), getMessageTimestamp(checkpointTimestamp, 1300)), - MessageData("D", "4".toValue(), getMessageTimestamp(checkpointTimestamp, 1500)), - MessageData("E", "5".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)), - // should be skipped because of message timeout - MessageData("F", "6".toValue(), getMessageTimestamp(checkpointTimestamp, 1600)) - ) - ) - - val exception = assertThrows { - noMessageCheckTask( - createEvent("root"), - streams, - createPreFilter("E", "5", FilterOperation.EQUAL), - TaskTimeout(1000, 5000) - ).begin(createCheckpoint(checkpointTimestamp, 1)) - } - assertEquals("The 'timeout' should be greater than the 'message timeout'", exception.message) - } - private fun createMessages( vararg messageData: MessageData, @@ -271,15 +211,6 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { ) } - companion object { - @JvmStatic - fun taskTimeouts(): Stream { - return Stream.of( - Arguments.arguments(TaskTimeout(5000, 1500)), // with timeout and message timeout - Arguments.arguments(TaskTimeout(0, 1500)) // with message timeout and missed timeout - ) - } - } data class MessageData(val fieldName: String, val value: Value, val timestamp: Timestamp) } \ No newline at end of file From 7392578868f2c17e7731d8a0d6ec77df6b994eb9 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Wed, 6 Oct 2021 16:46:51 +0300 Subject: [PATCH 18/41] [TH2-1950] Optimize the prefilter operation --- .../th2/check1/rule/AbstractCheckTask.kt | 2 +- .../th2/check1/rule/MessageContainer.kt | 11 +++++- .../exactpro/th2/check1/rule/StreamUtils.kt | 38 +++++++++++++++++++ .../rule/nomessage/NoMessageCheckTask.kt | 16 ++------ .../rule/sequence/SequenceCheckRuleTask.kt | 14 ++----- 5 files changed, 55 insertions(+), 26 deletions(-) create mode 100644 src/main/kotlin/com/exactpro/th2/check1/rule/StreamUtils.kt diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 9c3daedf..f5d17a1c 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -463,7 +463,7 @@ abstract class AbstractCheckTask( ) } - protected fun matchFilter( + internal fun matchFilter( messageContainer: MessageContainer, messageFilter: SailfishFilter, metadataFilter: SailfishFilter?, diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/MessageContainer.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/MessageContainer.kt index 2a8fe5e1..d5865160 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/MessageContainer.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/MessageContainer.kt @@ -12,6 +12,7 @@ */ package com.exactpro.th2.check1.rule +import com.exactpro.sf.common.impl.messages.DefaultMessageFactory import com.exactpro.sf.common.messages.IMessage import com.exactpro.sf.comparison.ComparatorSettings import com.exactpro.sf.comparison.ComparisonResult @@ -25,9 +26,17 @@ class MessageContainer( val metadataMessage: IMessage by lazy { VerificationUtil.toMessage(protoMessage.metadata) } + + companion object { + private val EMPTY_MESSAGE = DefaultMessageFactory.getFactory().createMessage("empty", "empty") + @JvmField + val FAKE = MessageContainer(Message.getDefaultInstance(), EMPTY_MESSAGE) + } } class SailfishFilter( val message: IMessage, val comparatorSettings: ComparatorSettings -) \ No newline at end of file +) + +fun MessageContainer.isNotFake(): Boolean = this !== MessageContainer.FAKE \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/StreamUtils.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/StreamUtils.kt new file mode 100644 index 00000000..5507219f --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/StreamUtils.kt @@ -0,0 +1,38 @@ +/* + * Copyright 2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.rule + +import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.message.toJson +import io.reactivex.Observable +import org.slf4j.Logger + +fun AbstractCheckTask.preFilterBy( + stream: Observable, + protoPreMessageFilter: RootMessageFilter, + messagePreFilter: SailfishFilter, + metadataPreFilter: SailfishFilter?, + logger: Logger, + onMatch: (ComparisonContainer) -> Unit +): Observable = + stream.map { messageContainer -> // Compare the message with pre-filter + if (logger.isDebugEnabled) { + logger.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) + } + val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false, significant = false) + ComparisonContainer(messageContainer, protoPreMessageFilter, result) + .takeIf(ComparisonContainer::fullyMatches) + ?.also(onMatch) + ?.messageContainer ?: MessageContainer.FAKE + }.filter(MessageContainer::isNotFake) \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index ff7cb36f..25d619a2 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -18,9 +18,9 @@ import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask -import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter +import com.exactpro.th2.check1.rule.preFilterBy import com.exactpro.th2.check1.util.VerificationUtil import com.exactpro.th2.check1.utils.toRootMessageFilter import com.exactpro.th2.common.event.Event @@ -28,7 +28,6 @@ import com.exactpro.th2.common.event.EventUtils import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.RootMessageFilter -import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toTreeTable import com.exactpro.th2.common.schema.message.MessageRouter import io.reactivex.Observable @@ -76,22 +75,13 @@ class NoMessageCheckTask( rootEvent.addSubEvent(resultEvent) } - //TODO: Need to extract logic from here and from the SequenceCheck rule to some utility class and try to optimize that logic override fun Observable.taskPipeline(): Observable = - map { messageContainer -> // Compare the message with pre-filter - if (LOGGER.isDebugEnabled) { - LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) - } - val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false) - ComparisonContainer(messageContainer, protoPreMessageFilter, result) - }.filter { preFilterContainer -> // Filter check result of pre-filter - preFilterContainer.fullyMatches - }.doOnNext { preFilterContainer -> // Update pre-filter state + preFilterBy(this, protoPreMessageFilter, messagePreFilter, metadataPreFilter, LOGGER) { preFilterContainer -> // Update pre-filter state with(preFilterContainer) { preFilterEvent.appendEventsWithVerification(preFilterContainer) preFilterEvent.messageID(protoActual.metadata.id) } - }.map(ComparisonContainer::messageContainer) + } override fun name(): String = "No message check" diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index edb47bc1..7ece4bb8 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -26,6 +26,7 @@ import com.exactpro.th2.check1.rule.AggregatedFilterResult import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter +import com.exactpro.th2.check1.rule.preFilterBy import com.exactpro.th2.check1.util.VerificationUtil import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.event.Event.Status.FAILED @@ -38,7 +39,6 @@ import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.MessageID import com.exactpro.th2.common.grpc.RootMessageFilter -import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toReadableBodyCollection import com.exactpro.th2.common.schema.message.MessageRouter import com.google.protobuf.TextFormat.shortDebugString @@ -124,22 +124,14 @@ class SequenceCheckRuleTask( } override fun Observable.taskPipeline(): Observable = - map { messageContainer -> // Compare the message with pre-filter - if (LOGGER.isDebugEnabled) { - LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) - } - val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false, significant = false) - ComparisonContainer(messageContainer, protoPreMessageFilter, result) - }.filter { preFilterContainer -> // Filter check result of pre-filter - preFilterContainer.fullyMatches - }.doOnNext { preFilterContainer -> // Update pre-filter state + preFilterBy(this, protoPreMessageFilter, messagePreFilter, metadataPreFilter, LOGGER) { preFilterContainer -> // Update pre-filter state with(preFilterContainer) { preFilterEvent.appendEventsWithVerification(preFilterContainer) preFilterEvent.messageID(protoActual.metadata.id) preFilteringResults[protoActual.metadata.id] = preFilterContainer } - }.map(ComparisonContainer::messageContainer) + } override fun onNext(messageContainer: MessageContainer) { for (index in messageFilters.indices) { From 3b50d7a139d93e9f4faa5e3022b821c38f056db5 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Thu, 30 Sep 2021 20:05:13 +0300 Subject: [PATCH 19/41] [TH2-2437] Add auto silence check for check sequence rule --- README.md | 11 +- build.gradle | 3 +- .../configuration/Check1Configuration.java | 9 + .../exactpro/th2/check1/CollectorService.kt | 9 + .../th2/check1/rule/AbstractCheckTask.kt | 57 ++++- .../exactpro/th2/check1/rule/RuleFactory.kt | 56 ++++- .../check1/rule/sequence/SilenceCheckTask.kt | 152 ++++++++++++ .../th2/check1/rule/AbstractCheckTaskTest.kt | 3 +- .../TestSequenceCheckTaskWithSilenceCheck.kt | 234 ++++++++++++++++++ 9 files changed, 507 insertions(+), 27 deletions(-) create mode 100644 src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt create mode 100644 src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt diff --git a/README.md b/README.md index 479e38e3..4d67a2e2 100644 --- a/README.md +++ b/README.md @@ -36,6 +36,7 @@ spec: cleanup-older-than: '60' cleanup-time-unit: 'SECONDS' max-event-batch-content-size: '1048576' + auto-sequence-rule-silence-check: false type: th2-check1 pins: - name: server @@ -69,7 +70,8 @@ This block describes the configuration for check1. "cleanup-older-than": 60, "cleanup-time-unit": "SECONDS", "max-event-batch-content-size": "1048576", - "rule-execution-timeout": 5000 + "rule-execution-timeout": 5000, + "auto-sequence-rule-silence-check": false } ``` @@ -88,6 +90,9 @@ The time unit for _cleanup-older-than_ setting. The available values are MILLIS, #### max-event-batch-content-size The max size in bytes of summary events content in a batch defined in _max-event-batch-content-size_ setting. _The default value is set to 1048576_ +#### auto-sequence-rule-silence-check +Defines a default behavior for creating CheckSequenceRule if `silence_check` parameter is not specified in the request. The default value is `false` + ## Required pins The Check1 component has two types of pin: @@ -128,6 +133,10 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type #### Added: + Implemented NoMessageCheck rule task. Updated CheckRule and CheckSequence rule tasks + New configuration parameter `rule-execution-timeout` witch is used if the user has not specified a timeout for the rule execution ++ Auto silence check after the CheckSequenceRule. + It verifies that there were not any messages matches the pre-filter in original request for CheckSequenceRule. + It awaits for realtime timeout that is equal to clean-up timeout. + Report about unexpected messages only after the timeout exceeds. Reports nothing if any task is added to the chain. ### 3.8.0 diff --git a/build.gradle b/build.gradle index 49c925dc..7e134718 100644 --- a/build.gradle +++ b/build.gradle @@ -42,6 +42,7 @@ repositories { name 'Sonatype_releases' url 'https://s01.oss.sonatype.org/content/repositories/releases/' } + mavenLocal() configurations.all { resolutionStrategy.cacheChangingModulesFor 0, 'seconds' @@ -165,7 +166,7 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.4.2' + implementation 'com.exactpro.th2:grpc-check1:3.5.0' implementation 'com.exactpro.th2:common:3.26.4' implementation 'com.exactpro.th2:sailfish-utils:3.9.1' implementation "org.slf4j:slf4j-log4j12" diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index 28073d83..4642757c 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -16,6 +16,7 @@ import java.time.temporal.ChronoUnit; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class Check1Configuration { @@ -34,6 +35,10 @@ public class Check1Configuration { @JsonProperty(value="rule-execution-timeout", defaultValue = "5000") private long ruleExecutionTimeout = 5000L; + @JsonProperty("auto-sequence-rule-silence-check") + @JsonPropertyDescription("The default behavior in case the SequenceCheckRule does not have silenceCheck parameter specified") + private boolean autoSequenceRuleSilenceCheck; + public int getMessageCacheSize() { return messageCacheSize; } @@ -53,4 +58,8 @@ public ChronoUnit getCleanupTimeUnit() { public long getRuleExecutionTimeout() { return ruleExecutionTimeout; } + + public boolean isAutoSequenceRuleSilenceCheck() { + return autoSequenceRuleSilenceCheck; + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index e14bf6c6..45be1916 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -62,6 +62,7 @@ class CollectorService( private val olderThanDelta = configuration.cleanupOlderThan private val olderThanTimeUnit = configuration.cleanupTimeUnit + private val defaultAutoSilenceCheck: Boolean = configuration.isAutoSequenceRuleSilenceCheck private var ruleFactory: RuleFactory @@ -105,9 +106,17 @@ class CollectorService( val task = ruleFactory.createSequenceCheckRule(request) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) + val silenceCheck = if (request.hasSilenceCheck()) request.silenceCheck.value else defaultAutoSilenceCheck + + val silenceCheckTask: AbstractCheckTask? = if (silenceCheck) { + ruleFactory.createSilenceCheck(request, olderThanTimeUnit.duration.toMillis() * olderThanDelta) + } else { + null + } eventIdToLastCheckTask.compute(CheckTaskKey(chainID, request.connectivityId)) { _, value -> task.apply { addToChainOrBegin(value, request.checkpoint) } + .run { silenceCheckTask?.also { subscribeNextTask(it) } ?: this } } return chainID } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index f5d17a1c..fc51e1c1 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -99,6 +99,13 @@ abstract class AbstractCheckTask( private val taskState = AtomicReference(State.CREATED) @Volatile private var streamCompletedState = State.STREAM_COMPLETED + @Volatile + private var completed = false + @Volatile + protected var isCanceled = false + private set + protected var isParentCompleted: Boolean? = null + private set /** * Used for observe messages in one thread. @@ -171,17 +178,18 @@ abstract class AbstractCheckTask( */ fun subscribeNextTask(checkTask: AbstractCheckTask) { if (hasNextTask.compareAndSet(false, true)) { + onChainedTaskSubscription() sequenceSubject.subscribe { legacy -> val executor = if (legacy.executorService.isShutdown) { - LOGGER.warn("Executor has been shutdown before next task has been subscribed. Create a new one") - createExecutorService() - } else { - legacy.executorService - } + LOGGER.warn("Executor has been shutdown before next task has been subscribed. Create a new one") + createExecutorService() + } else { + legacy.executorService + } legacy.sequenceData.apply { - checkTask.begin(lastSequence, lastMessageTimestamp, executor, untrusted) + checkTask.begin(lastSequence, lastMessageTimestamp, executor, untrusted, completed) } - } + } LOGGER.info("Task {} ({}) subscribed to task {} ({})", checkTask.description, checkTask.hashCode(), description, hashCode()) } else { throw IllegalStateException("Subscription to last sequence for task $description (${hashCode()}) is already executed, subscriber ${checkTask.description} (${checkTask.hashCode()})") @@ -200,6 +208,11 @@ abstract class AbstractCheckTask( begin(checkpointData?.sequence ?: DEFAULT_SEQUENCE, checkpointData?.timestamp) } + /** + * Callback when another task is subscribed to the result of the current task + */ + protected open fun onChainedTaskSubscription() {} + /** * It is called when the timeout is over and the task is not complete yet */ @@ -216,6 +229,7 @@ abstract class AbstractCheckTask( val prevValue = taskState.getAndSet(State.TASK_COMPLETED) dispose() endFuture.dispose() + completed = true when (prevValue) { State.TIMEOUT -> { @@ -238,6 +252,12 @@ abstract class AbstractCheckTask( */ protected open fun Observable.taskPipeline() : Observable = this + /** + * @return `true` if another task has been subscribed to the result of the current task. + * Otherwise, returns `false` + */ + protected fun hasNextTask(): Boolean = hasNextTask.get() + protected abstract fun name(): String protected abstract fun type(): String protected abstract fun setup(rootEvent: Event) @@ -250,11 +270,18 @@ abstract class AbstractCheckTask( * @param checkpointTimestamp checkpoint timestamp from the previous task * @param executorService executor to schedule pipeline execution. * @param untrusted flag is guarantee that the previous sequence data is correct + * @param parentTaskCompleted indicates whether the parent task was completed normally. `null` if no parent task exists. * @throws IllegalStateException when method is called more than once. */ - private fun begin(sequence: Long = DEFAULT_SEQUENCE, checkpointTimestamp: Timestamp? = null, - executorService: ExecutorService = createExecutorService(), untrusted: Boolean = false) { + private fun begin( + sequence: Long = DEFAULT_SEQUENCE, + checkpointTimestamp: Timestamp? = null, + executorService: ExecutorService = createExecutorService(), + untrusted: Boolean = false, + parentTaskCompleted: Boolean? = null + ) { configureRootEvent() + isParentCompleted = parentTaskCompleted if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } @@ -307,6 +334,10 @@ abstract class AbstractCheckTask( } } + protected fun cancel() { + isCanceled = true + } + private fun taskFinished() { try { val currentState = taskState.get() @@ -314,8 +345,12 @@ abstract class AbstractCheckTask( if (currentState.callOnTimeoutCallback) { callOnTimeoutCallback() } - publishEvent() - LOGGER.info("Task '$description' has been finished") + if (isCanceled) { + LOGGER.info("Task '$description' has been canceled. No result will published") + } else { + publishEvent() + LOGGER.info("Task '$description' has been finished") + } } catch (ex: Exception) { val message = "Cannot finish task '$description'" LOGGER.error(message, ex) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt index 992c756e..24e9473e 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt @@ -22,9 +22,11 @@ import com.exactpro.th2.check1.exception.RuleInternalException import com.exactpro.th2.check1.grpc.CheckRuleRequest import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest import com.exactpro.th2.check1.grpc.NoMessageCheckRequest +import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.check.CheckRuleTask import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask +import com.exactpro.th2.check1.rule.sequence.SilenceCheckTask import com.exactpro.th2.common.event.Event import com.exactpro.th2.common.grpc.Checkpoint import com.exactpro.th2.common.grpc.ComparisonSettings @@ -52,8 +54,8 @@ class RuleFactory( private val defaultRuleExecutionTimeout = configuration.ruleExecutionTimeout fun createCheckRule(request: CheckRuleRequest): CheckRuleTask = - ruleCreation(request, request.parentEventId) { - checkAndCreateRule { request -> + ruleCreation(request.parentEventId) { + checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias @@ -90,8 +92,8 @@ class RuleFactory( } fun createSequenceCheckRule(request: CheckSequenceRuleRequest): SequenceCheckRuleTask = - ruleCreation(request, request.parentEventId) { - checkAndCreateRule { request -> + ruleCreation(request.parentEventId) { + checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias @@ -129,8 +131,8 @@ class RuleFactory( } fun createNoMessageCheckRule(request: NoMessageCheckRequest): NoMessageCheckTask = - ruleCreation(request, request.parentEventId) { - checkAndCreateRule { request -> + ruleCreation(request.parentEventId) { + checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } val parentEventID: EventID = request.parentEventId check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } @@ -157,11 +159,39 @@ class RuleFactory( } } + fun createSilenceCheck( + request: CheckSequenceRuleRequest, + timeout: Long + ): SilenceCheckTask { + return ruleCreation(request.parentEventId) { + checkAndCreateRule { + check(timeout > 0) { "timeout must be greater that zero" } + val sessionAlias: String = request.connectivityId.sessionAlias + val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) + SilenceCheckTask( + request.preFilter, + request.description.takeIf(String::isNotEmpty), + createTaskTimeout(timeout), + maxEventBatchContentSize, + Instant.now(), + sessionKey, + request.parentEventId, + streamObservable, + eventBatchRouter + ) + } + onErrorEvent { + Event.start() + .name("Auto silence check rule cannot be created") + .type("checkRuleCreation") + } + } + } - private inline fun ruleCreation(request: T, parentEventId: EventID, block: RuleCreationContext.() -> Unit): R { - val ruleCreationContext = RuleCreationContext().apply(block) + private inline fun ruleCreation(parentEventId: EventID, block: RuleCreationContext.() -> Unit): R { + val ruleCreationContext = RuleCreationContext().apply(block) try { - return ruleCreationContext.action(request) + return ruleCreationContext.action() } catch (e: RuleInternalException) { throw e } catch (e: Exception) { @@ -238,7 +268,7 @@ class RuleFactory( } } - private fun createTaskTimeout(timeout: Long, messageTimeout: Long): TaskTimeout { + private fun createTaskTimeout(timeout: Long, messageTimeout: Long = 0): TaskTimeout { val newRuleTimeout = if (timeout <= 0) { LOGGER.info("Rule execution timeout is less than or equal to zero, used default rule execution timeout '$defaultRuleExecutionTimeout'") defaultRuleExecutionTimeout @@ -248,11 +278,11 @@ class RuleFactory( return TaskTimeout(newRuleTimeout, messageTimeout) } - private class RuleCreationContext { - lateinit var action: (T) -> R + private class RuleCreationContext { + lateinit var action: () -> R lateinit var event: () -> Event - fun checkAndCreateRule(block: (T) -> R) { + fun checkAndCreateRule(block: () -> R) { action = block } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt new file mode 100644 index 00000000..b65493a9 --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -0,0 +1,152 @@ +/* + * Copyright 2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.rule.sequence + +import com.exactpro.th2.check1.SessionKey +import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.TaskTimeout +import com.exactpro.th2.check1.grpc.PreFilter +import com.exactpro.th2.check1.rule.AbstractCheckTask +import com.exactpro.th2.check1.rule.ComparisonContainer +import com.exactpro.th2.check1.rule.MessageContainer +import com.exactpro.th2.check1.rule.SailfishFilter +import com.exactpro.th2.check1.util.VerificationUtil +import com.exactpro.th2.check1.utils.toRootMessageFilter +import com.exactpro.th2.common.event.Event +import com.exactpro.th2.common.event.EventUtils.createMessageBean +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventID +import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.message.toJson +import com.exactpro.th2.common.message.toReadableBodyCollection +import com.exactpro.th2.common.schema.message.MessageRouter +import io.reactivex.Observable +import java.time.Instant + +class SilenceCheckTask( + protoPreFilter: PreFilter, + description: String?, + taskTimeout: TaskTimeout, + maxEventBatchContentSize: Int, + submitTime: Instant, + sessionKey: SessionKey, + parentEventID: EventID, + messageStream: Observable, + eventBatchRouter: MessageRouter +) : AbstractCheckTask( + description, + taskTimeout, + maxEventBatchContentSize, + submitTime, + sessionKey, + parentEventID, + messageStream, + eventBatchRouter +) { + private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() + private val messagePreFilter = SailfishFilter( + converter.fromProtoPreFilter(protoPreMessageFilter), + protoPreMessageFilter.toCompareSettings() + ) + private val metadataPreFilter: SailfishFilter? = protoPreMessageFilter.metadataFilterOrNull()?.let { + SailfishFilter( + converter.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), + it.toComparisonSettings() + ) + } + private lateinit var preFilterEvent: Event + private lateinit var resultEvent: Event + private var extraMessagesCounter: Int = 0 + + @Volatile + private var started = false + + override fun onStart() { + super.onStart() + started = true + val hasNextTask = hasNextTask() + if (isParentCompleted == false || hasNextTask) { + if (hasNextTask) { + LOGGER.info("Has subscribed task. Skip checking extra messages") + } else { + LOGGER.info("Parent task was not finished normally. Skip checking extra messages") + } + cancel() + checkComplete() + return + } + preFilterEvent = Event.start() + .type("preFiltering") + .bodyData(protoPreMessageFilter.toReadableBodyCollection()) + + rootEvent.addSubEvent(preFilterEvent) + + resultEvent = Event.start() + .type("noMessagesCheckResult") + rootEvent.addSubEvent(resultEvent) + } + + override fun onChainedTaskSubscription() { + if (started) { + cancel() + checkComplete() + } else { + if (LOGGER.isInfoEnabled) { + LOGGER.info("The ${type()} task '$description' will be automatically canceled when it begins") + } + } + } + + override fun name(): String = "AutoSilenceCheck" + + override fun type(): String = "AutoSilenceCheck" + + override fun setup(rootEvent: Event) { + rootEvent.bodyData(createMessageBean("AutoSilenceCheck for session ${sessionKey.run { "$sessionAlias ($direction)" }}")) + } + + override fun Observable.taskPipeline(): Observable = + map { messageContainer -> // Compare the message with pre-filter + if (LOGGER.isDebugEnabled) { + LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) + } + val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false, significant = false) + ComparisonContainer(messageContainer, protoPreMessageFilter, result) + }.filter { preFilterContainer -> // Filter check result of pre-filter + preFilterContainer.fullyMatches + }.doOnNext { preFilterContainer -> // Update pre-filter state + with(preFilterContainer) { + preFilterEvent.appendEventsWithVerification(preFilterContainer) + preFilterEvent.messageID(protoActual.metadata.id) + } + }.map(ComparisonContainer::messageContainer) + + override fun onNext(container: MessageContainer) { + container.protoMessage.metadata.apply { + extraMessagesCounter++ + resultEvent.messageID(id) + } + } + + override fun completeEvent(taskState: State) { + preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") + + if (extraMessagesCounter == 0) { + resultEvent.status(Event.Status.PASSED).name("Check passed") + } else { + resultEvent.status(Event.Status.FAILED) + .name("Check failed: $extraMessagesCounter extra messages were found.") + } + } +} \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index a6bee296..695a7b7f 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -15,6 +15,7 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.grpc.PreFilter +import com.exactpro.th2.common.event.EventUtils import com.exactpro.th2.common.event.IBodyData import com.exactpro.th2.common.event.bean.Verification import com.exactpro.th2.common.event.bean.VerificationEntry @@ -77,7 +78,7 @@ abstract class AbstractCheckTaskTest { } protected fun createEvent(id: String): EventID { - return EventID.newBuilder().setId(id).build() + return requireNotNull(EventUtils.toEventID(id)) } protected fun getMessageTimestamp(start: Instant, delta: Long): Timestamp = diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt new file mode 100644 index 00000000..d186adc9 --- /dev/null +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt @@ -0,0 +1,234 @@ +/* + * Copyright 2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.rule.sequence + +import com.exactpro.th2.check1.configuration.Check1Configuration +import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest +import com.exactpro.th2.check1.rule.AbstractCheckTaskTest +import com.exactpro.th2.check1.rule.RuleFactory +import com.exactpro.th2.common.grpc.ConnectionID +import com.exactpro.th2.common.grpc.Direction +import com.exactpro.th2.common.grpc.EventBatch +import com.exactpro.th2.common.grpc.EventStatus +import com.exactpro.th2.common.grpc.FilterOperation +import com.exactpro.th2.common.grpc.ValueFilter +import com.exactpro.th2.common.message.messageFilter +import com.exactpro.th2.common.message.rootMessageFilter +import com.exactpro.th2.common.message.toJson +import com.exactpro.th2.common.value.toValue +import org.junit.jupiter.api.Assertions.assertEquals +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertAll +import kotlin.test.assertNull + +class TestSequenceCheckTaskWithSilenceCheck : AbstractCheckTaskTest() { + + @Test + fun `reports about extra messages when timeout exceeded`() { + val streams = createStreams(messages = (0..2L).map { + constructMessage(sequence = it) + .putFields("A", 42.toValue()) + .putFields("B", it.toValue()) + .build() + }) + val factory = RuleFactory(Check1Configuration(), streams, clientStub) + + val filters = (0..1).map { + rootMessageFilter(MESSAGE_TYPE) + .setMessageFilter(messageFilter() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("42").setKey(true).build()) + .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter(it.toString()).build()) + ).build() + } + val preFilter = createPreFilter("A", "42", FilterOperation.EQUAL) + val parentId = createEvent("root") + + val request = CheckSequenceRuleRequest.newBuilder() + .setConnectivityId(ConnectionID.newBuilder().setSessionAlias(SESSION_ALIAS)) + .setDirection(Direction.FIRST) + .setTimeout(1000) + .addAllRootMessageFilters(filters) + .setPreFilter(preFilter) + .setParentEventId(parentId) + .build() + val sequenceRule = factory.createSequenceCheckRule(request) + val silenceCheck = factory.createSilenceCheck(request, 1000) + sequenceRule.subscribeNextTask(silenceCheck) + sequenceRule.begin() + + val events = awaitEventBatchRequest(2000, 10).flatMap(EventBatch::getEventsList) + val silenceCheckRoot = events.first { it.type == "AutoSilenceCheck" }.id + val result = events.first { it.type == "noMessagesCheckResult" && it.parentId == silenceCheckRoot } + assertAll( + { assertEquals(EventStatus.FAILED, result.status) { "Unexpected status for event: ${result.toJson()}" } }, + { assertEquals("Check failed: 1 extra messages were found.", result.name) { "Unexpected name for event: ${result.toJson()}" } }, + { + assertEquals(listOf(2L), result.attachedMessageIdsList.map { it.sequence }) { + "Unexpected messages attached: ${result.attachedMessageIdsList.map { it.toJson() }}" + } + } + ) + } + + @Test + fun `reports no extra messages found when timeout exceeded`() { + val streams = createStreams(messages = (0..1L).map { + constructMessage(sequence = it) + .putFields("A", 42.toValue()) + .putFields("B", it.toValue()) + .build() + }) + val factory = RuleFactory(Check1Configuration(), streams, clientStub) + + val filters = (0..1).map { + rootMessageFilter(MESSAGE_TYPE) + .setMessageFilter(messageFilter() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("42").setKey(true).build()) + .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter(it.toString()).build()) + ).build() + } + val preFilter = createPreFilter("A", "42", FilterOperation.EQUAL) + val parentId = createEvent("root") + + val request = CheckSequenceRuleRequest.newBuilder() + .setConnectivityId(ConnectionID.newBuilder().setSessionAlias(SESSION_ALIAS)) + .setDirection(Direction.FIRST) + .setTimeout(1000) + .addAllRootMessageFilters(filters) + .setPreFilter(preFilter) + .setParentEventId(parentId) + .build() + val sequenceRule = factory.createSequenceCheckRule(request) + val silenceCheck = factory.createSilenceCheck(request, 1000) + sequenceRule.subscribeNextTask(silenceCheck) + sequenceRule.begin() + + val events = awaitEventBatchRequest(2000, 8).flatMap(EventBatch::getEventsList) + val silenceCheckRoot = events.first { it.type == "AutoSilenceCheck" }.id + val result = events.first { it.type == "noMessagesCheckResult" && it.parentId == silenceCheckRoot } + assertAll( + { assertEquals(EventStatus.SUCCESS, result.status) { "Unexpected status for event: ${result.toJson()}" } }, + { assertEquals("Check passed", result.name) { "Unexpected name for event: ${result.toJson()}" } } + ) + } + + @Test + fun `does not report if next rule is subscribed in chain before beginning`() { + val streams = createStreams(messages = (0..2L).map { + constructMessage(sequence = it) + .putFields("A", 42.toValue()) + .putFields("B", it.toValue()) + .build() + }) + val factory = RuleFactory(Check1Configuration(), streams, clientStub) + + val filters = (0..1).map { + rootMessageFilter(MESSAGE_TYPE) + .setMessageFilter(messageFilter() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("42").setKey(true).build()) + .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter(it.toString()).build()) + ).build() + } + val preFilter = createPreFilter("A", "42", FilterOperation.EQUAL) + val parentId = createEvent("root") + + val request = CheckSequenceRuleRequest.newBuilder() + .setConnectivityId(ConnectionID.newBuilder().setSessionAlias(SESSION_ALIAS)) + .setDescription("1") + .setDirection(Direction.FIRST) + .setTimeout(1000) + .addAllRootMessageFilters(filters) + .setPreFilter(preFilter) + .setParentEventId(parentId) + .build() + val anotherRequest = request.toBuilder() + .clearRootMessageFilters() + .setDescription("2") + .addRootMessageFilters(rootMessageFilter(MESSAGE_TYPE) + .setMessageFilter(messageFilter() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("42").setKey(true).build()) + .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("2").build()) + ).build()) + .build() + val sequenceRule = factory.createSequenceCheckRule(request) + val silenceCheck = factory.createSilenceCheck(request, 1000) + val anotherRule = factory.createSequenceCheckRule(anotherRequest) + sequenceRule.subscribeNextTask(silenceCheck) + sequenceRule.begin() + silenceCheck.subscribeNextTask(anotherRule) + + val events = awaitEventBatchRequest(2000, 12).flatMap(EventBatch::getEventsList) + assertAll( + { assertNull(events.find { it.type == "AutoSilenceCheck" }, "Unexpected events: $events") }, + { + events.filter { it.name == "Check sequence rule - 1" } + .also { + assertEquals(1, it.size) { "Unexpected count of events for the first rule: $it" } + } + }, + { + events.filter { it.name == "Check sequence rule - 2" } + .also { + assertEquals(1, it.size) { "Unexpected count of events for the second rule: $it" } + } + } + ) + } + + @Test + fun `does not report when next rule added to the chain before timeout exceeds`() { + val streams = createStreams(messages = (0..1L).map { + constructMessage(sequence = it) + .putFields("A", 42.toValue()) + .putFields("B", it.toValue()) + .build() + }) + val factory = RuleFactory(Check1Configuration(), streams, clientStub) + + val filters = (0..1).map { + rootMessageFilter(MESSAGE_TYPE) + .setMessageFilter(messageFilter() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("42").setKey(true).build()) + .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter(it.toString()).build()) + ).build() + } + val preFilter = createPreFilter("A", "42", FilterOperation.EQUAL) + val parentId = createEvent("root") + + val request = CheckSequenceRuleRequest.newBuilder() + .setConnectivityId(ConnectionID.newBuilder().setSessionAlias(SESSION_ALIAS)) + .setDescription("1") + .setDirection(Direction.FIRST) + .setTimeout(1000) + .addAllRootMessageFilters(filters) + .setPreFilter(preFilter) + .setParentEventId(parentId) + .build() + val silenceCheck = factory.createSilenceCheck(request, 1000) + val sequenceRule = factory.createSequenceCheckRule(request.toBuilder().setDescription("2").build()) + silenceCheck.begin() + silenceCheck.subscribeNextTask(sequenceRule) + + val events = awaitEventBatchRequest(2000, 6).flatMap(EventBatch::getEventsList) + assertAll( + { assertNull(events.find { it.type == "AutoSilenceCheck" }, "Unexpected events: $events") }, + { + events.filter { it.name == "Check sequence rule - 2" } + .also { + assertEquals(1, it.size) { "Unexpected count of events for the rule: $it" } + } + } + ) + } +} \ No newline at end of file From 74c24c346406648e177887ba8bb67a33d8843e1f Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 1 Oct 2021 11:22:16 +0300 Subject: [PATCH 20/41] [TH2-2437] Dev build for grpc-check1 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 7e134718..04096c63 100644 --- a/build.gradle +++ b/build.gradle @@ -166,7 +166,7 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0' + implementation 'com.exactpro.th2:grpc-check1:3.5.0-th2-2437-1292014101-SNAPSHOT' implementation 'com.exactpro.th2:common:3.26.4' implementation 'com.exactpro.th2:sailfish-utils:3.9.1' implementation "org.slf4j:slf4j-log4j12" From 57eee8dd427bce565caec6ea48269ef950dfc1df Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 1 Oct 2021 19:27:39 +0300 Subject: [PATCH 21/41] [TH2-2437] Move cancelation information to the implementation level. Correct readme --- README.md | 16 ++++++---- .../configuration/Check1Configuration.java | 8 ++--- .../exactpro/th2/check1/CollectorService.kt | 2 +- .../th2/check1/rule/AbstractCheckTask.kt | 32 ++++++++----------- .../rule/nomessage/NoMessageCheckTask.kt | 3 +- .../rule/sequence/SequenceCheckRuleTask.kt | 3 +- .../check1/rule/sequence/SilenceCheckTask.kt | 20 +++++++++--- 7 files changed, 47 insertions(+), 37 deletions(-) diff --git a/README.md b/README.md index 4d67a2e2..39a7102c 100644 --- a/README.md +++ b/README.md @@ -6,7 +6,7 @@ The component is responsible for verifying decoded messages. Communication with the script takes place via grpc, messages are received via rabbit mq. -The component subscribes to queues specified in the configuration and accumulates messages from them in a FIFO buffer. +The component subscribes to the queues specified in the configuration and accumulates messages from them in a FIFO buffer. The buffer size is configurable, and it is set to 1000 by default. @@ -17,6 +17,10 @@ When the component starts, the grpc server also starts and then the component wa Available requests are described in [this repository](https://gitlab.exactpro.com/vivarium/th2/th2-core-open-source/th2-grpc-check1) - CheckSequenceRuleRequest - prefilters the messages and verify all of them by filter. Order checking configured from request. + Depending on the request and check1 configuration **SilenceCheckRule** can be added after the CheckSequenceRule. + It verifies that there were not any messages matches the pre-filter in original request. + It awaits for realtime timeout that is equal to clean-up timeout. + Report about unexpected messages only after the timeout exceeds. Reports nothing if any task is added to the chain. - CheckRuleRequest - get message filter from request and check it with messages in the cache or await specified time in case of empty cache or message absence. - NoMessageCheckRequest - prefilters messages and verifies that no other messages have been received. @@ -36,7 +40,7 @@ spec: cleanup-older-than: '60' cleanup-time-unit: 'SECONDS' max-event-batch-content-size: '1048576' - auto-sequence-rule-silence-check: false + auto-silence-check-after-sequence-rule: false type: th2-check1 pins: - name: server @@ -71,7 +75,7 @@ This block describes the configuration for check1. "cleanup-time-unit": "SECONDS", "max-event-batch-content-size": "1048576", "rule-execution-timeout": 5000, - "auto-sequence-rule-silence-check": false + "auto-silence-check-after-sequence-rule": false } ``` @@ -90,7 +94,7 @@ The time unit for _cleanup-older-than_ setting. The available values are MILLIS, #### max-event-batch-content-size The max size in bytes of summary events content in a batch defined in _max-event-batch-content-size_ setting. _The default value is set to 1048576_ -#### auto-sequence-rule-silence-check +#### auto-silence-check-after-sequence-rule Defines a default behavior for creating CheckSequenceRule if `silence_check` parameter is not specified in the request. The default value is `false` ## Required pins @@ -134,9 +138,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Implemented NoMessageCheck rule task. Updated CheckRule and CheckSequence rule tasks + New configuration parameter `rule-execution-timeout` witch is used if the user has not specified a timeout for the rule execution + Auto silence check after the CheckSequenceRule. - It verifies that there were not any messages matches the pre-filter in original request for CheckSequenceRule. - It awaits for realtime timeout that is equal to clean-up timeout. - Report about unexpected messages only after the timeout exceeds. Reports nothing if any task is added to the chain. ++ `auto-silence-check-after-sequence-rule` to setup a default behavior for CheckSequenceRule ### 3.8.0 diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index 4642757c..0841df51 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -35,9 +35,9 @@ public class Check1Configuration { @JsonProperty(value="rule-execution-timeout", defaultValue = "5000") private long ruleExecutionTimeout = 5000L; - @JsonProperty("auto-sequence-rule-silence-check") + @JsonProperty("auto-silence-check-after-sequence-rule") @JsonPropertyDescription("The default behavior in case the SequenceCheckRule does not have silenceCheck parameter specified") - private boolean autoSequenceRuleSilenceCheck; + private boolean autoSilenceCheckAfterSequenceRule; public int getMessageCacheSize() { return messageCacheSize; @@ -59,7 +59,7 @@ public long getRuleExecutionTimeout() { return ruleExecutionTimeout; } - public boolean isAutoSequenceRuleSilenceCheck() { - return autoSequenceRuleSilenceCheck; + public boolean isAutoSilenceCheckAfterSequenceRule() { + return autoSilenceCheckAfterSequenceRule; } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 45be1916..95d59fa6 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -62,7 +62,7 @@ class CollectorService( private val olderThanDelta = configuration.cleanupOlderThan private val olderThanTimeUnit = configuration.cleanupTimeUnit - private val defaultAutoSilenceCheck: Boolean = configuration.isAutoSequenceRuleSilenceCheck + private val defaultAutoSilenceCheck: Boolean = configuration.isAutoSilenceCheckAfterSequenceRule private var ruleFactory: RuleFactory diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index fc51e1c1..762bde34 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -101,9 +101,6 @@ abstract class AbstractCheckTask( private var streamCompletedState = State.STREAM_COMPLETED @Volatile private var completed = false - @Volatile - protected var isCanceled = false - private set protected var isParentCompleted: Boolean? = null private set @@ -334,10 +331,6 @@ abstract class AbstractCheckTask( } } - protected fun cancel() { - isCanceled = true - } - private fun taskFinished() { try { val currentState = taskState.get() @@ -345,12 +338,8 @@ abstract class AbstractCheckTask( if (currentState.callOnTimeoutCallback) { callOnTimeoutCallback() } - if (isCanceled) { - LOGGER.info("Task '$description' has been canceled. No result will published") - } else { - publishEvent() - LOGGER.info("Task '$description' has been finished") - } + publishEvent() + LOGGER.info("Task '$description' has been finished") } catch (ex: Exception) { val message = "Cannot finish task '$description'" LOGGER.error(message, ex) @@ -405,8 +394,9 @@ abstract class AbstractCheckTask( /** * Prepare the root event or children events for publication. * This method is invoked in [State.PUBLISHED] state. + * @return `true` if the event should be published. Otherwise, `false` */ - protected open fun completeEvent(taskState: State) {} + protected open fun completeEvent(taskState: State): Boolean = true protected fun isCheckpointLastReceivedMessage(): Boolean = bufferContainsStartMessage && !hasMessagesInTimeoutInterval @@ -416,9 +406,13 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - completeEventOrReportError(prevState) + val publish = completeEventOrReportError(prevState) _endTime = Instant.now() + if (!publish) { + LOGGER.info("Skip event publication for task ${type()} '$description' (${hashCode()})") + return + } val batches = rootEvent.disperseToBatches(maxEventBatchContentSize, parentEventID) RESPONSE_EXECUTOR.execute { @@ -439,10 +433,9 @@ abstract class AbstractCheckTask( } } - private fun completeEventOrReportError(prevState: State) { - try { - completeEvent(prevState) - doAfterCompleteEvent() + private fun completeEventOrReportError(prevState: State): Boolean { + return try { + completeEvent(prevState).also { doAfterCompleteEvent() } } catch (e: Exception) { LOGGER.error("Result event cannot be completed", e) rootEvent.addSubEventWithSamePeriod() @@ -451,6 +444,7 @@ abstract class AbstractCheckTask( .bodyData(EventUtils.createMessageBean("An unexpected exception has been thrown during result check build")) .bodyData(EventUtils.createMessageBean(e.message)) .status(FAILED) + true } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 25d619a2..841bae31 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -98,7 +98,7 @@ class NoMessageCheckTask( } } - override fun completeEvent(taskState: State) { + override fun completeEvent(taskState: State): Boolean { preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") if (extraMessagesCounter == 0) { @@ -117,5 +117,6 @@ class NoMessageCheckTask( } resultEvent.addSubEvent(executionStopEvent) } + return super.completeEvent(taskState) } } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 7ece4bb8..0f4f48d8 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -169,11 +169,12 @@ class SequenceCheckRuleTask( } } - override fun completeEvent(taskState: State) { + override fun completeEvent(taskState: State): Boolean { preFilterEvent.name("Pre-filtering (filtered ${preFilteringResults.size} / processed $handledMessageCounter) messages") fillSequenceEvent() fillCheckMessagesEvent() + return super.completeEvent(taskState) } override fun name(): String = "Check sequence rule" diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt index b65493a9..40d1d91b 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -33,6 +33,7 @@ import com.exactpro.th2.common.message.toReadableBodyCollection import com.exactpro.th2.common.schema.message.MessageRouter import io.reactivex.Observable import java.time.Instant +import java.util.concurrent.atomic.AtomicBoolean class SilenceCheckTask( protoPreFilter: PreFilter, @@ -71,6 +72,7 @@ class SilenceCheckTask( @Volatile private var started = false + private val isCanceled = AtomicBoolean() override fun onStart() { super.onStart() @@ -83,7 +85,6 @@ class SilenceCheckTask( LOGGER.info("Parent task was not finished normally. Skip checking extra messages") } cancel() - checkComplete() return } preFilterEvent = Event.start() @@ -98,9 +99,8 @@ class SilenceCheckTask( } override fun onChainedTaskSubscription() { - if (started) { + if (started) { // because we cannot cancel task before it is actually started cancel() - checkComplete() } else { if (LOGGER.isInfoEnabled) { LOGGER.info("The ${type()} task '$description' will be automatically canceled when it begins") @@ -139,7 +139,10 @@ class SilenceCheckTask( } } - override fun completeEvent(taskState: State) { + override fun completeEvent(taskState: State): Boolean { + if (isCanceled.get()) { + return false + } preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") if (extraMessagesCounter == 0) { @@ -148,5 +151,14 @@ class SilenceCheckTask( resultEvent.status(Event.Status.FAILED) .name("Check failed: $extraMessagesCounter extra messages were found.") } + return true + } + + private fun cancel() { + if (isCanceled.compareAndSet(false, true)) { + checkComplete() + } else { + LOGGER.debug("Task {} '{}' already canceled", type(), description) + } } } \ No newline at end of file From ae80de0c6331e865e6bd7baa7d1d3ed2371de9f8 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Wed, 6 Oct 2021 12:28:59 +0300 Subject: [PATCH 22/41] [TH2-2437] Create property for skipping publication --- .../exactpro/th2/check1/rule/AbstractCheckTask.kt | 13 ++++++++----- .../th2/check1/rule/nomessage/NoMessageCheckTask.kt | 3 +-- .../check1/rule/sequence/SequenceCheckRuleTask.kt | 3 +-- .../th2/check1/rule/sequence/SilenceCheckTask.kt | 10 ++++++---- 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 762bde34..02ea03d7 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -394,9 +394,10 @@ abstract class AbstractCheckTask( /** * Prepare the root event or children events for publication. * This method is invoked in [State.PUBLISHED] state. - * @return `true` if the event should be published. Otherwise, `false` */ - protected open fun completeEvent(taskState: State): Boolean = true + protected open fun completeEvent(taskState: State) {} + + protected open val skipPublication: Boolean = false protected fun isCheckpointLastReceivedMessage(): Boolean = bufferContainsStartMessage && !hasMessagesInTimeoutInterval @@ -406,10 +407,10 @@ abstract class AbstractCheckTask( private fun publishEvent() { val prevState = taskState.getAndSet(State.PUBLISHED) if (prevState != State.PUBLISHED) { - val publish = completeEventOrReportError(prevState) + val hasError = completeEventOrReportError(prevState) _endTime = Instant.now() - if (!publish) { + if (skipPublication && !hasError) { LOGGER.info("Skip event publication for task ${type()} '$description' (${hashCode()})") return } @@ -435,7 +436,9 @@ abstract class AbstractCheckTask( private fun completeEventOrReportError(prevState: State): Boolean { return try { - completeEvent(prevState).also { doAfterCompleteEvent() } + completeEvent(prevState) + doAfterCompleteEvent() + false } catch (e: Exception) { LOGGER.error("Result event cannot be completed", e) rootEvent.addSubEventWithSamePeriod() diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 841bae31..25d619a2 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -98,7 +98,7 @@ class NoMessageCheckTask( } } - override fun completeEvent(taskState: State): Boolean { + override fun completeEvent(taskState: State) { preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") if (extraMessagesCounter == 0) { @@ -117,6 +117,5 @@ class NoMessageCheckTask( } resultEvent.addSubEvent(executionStopEvent) } - return super.completeEvent(taskState) } } \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 0f4f48d8..7ece4bb8 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -169,12 +169,11 @@ class SequenceCheckRuleTask( } } - override fun completeEvent(taskState: State): Boolean { + override fun completeEvent(taskState: State) { preFilterEvent.name("Pre-filtering (filtered ${preFilteringResults.size} / processed $handledMessageCounter) messages") fillSequenceEvent() fillCheckMessagesEvent() - return super.completeEvent(taskState) } override fun name(): String = "Check sequence rule" diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt index 40d1d91b..40f38dba 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -139,9 +139,9 @@ class SilenceCheckTask( } } - override fun completeEvent(taskState: State): Boolean { - if (isCanceled.get()) { - return false + override fun completeEvent(taskState: State) { + if (skipPublication) { + return } preFilterEvent.name("Prefilter: $extraMessagesCounter messages were filtered.") @@ -151,9 +151,11 @@ class SilenceCheckTask( resultEvent.status(Event.Status.FAILED) .name("Check failed: $extraMessagesCounter extra messages were found.") } - return true } + override val skipPublication: Boolean + get() = isCanceled.get() + private fun cancel() { if (isCanceled.compareAndSet(false, true)) { checkComplete() From 9514070f4bc53bb85670caa4a3e7345db1478702 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 8 Oct 2021 13:18:43 +0300 Subject: [PATCH 23/41] [TH2-2437] Use utility function for pre-filtering --- build.gradle | 2 +- .../th2/check1/rule/sequence/SilenceCheckTask.kt | 15 +++------------ 2 files changed, 4 insertions(+), 13 deletions(-) diff --git a/build.gradle b/build.gradle index 04096c63..7e402ae1 100644 --- a/build.gradle +++ b/build.gradle @@ -166,7 +166,7 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0-th2-2437-1292014101-SNAPSHOT' + implementation 'com.exactpro.th2:grpc-check1:3.5.0-th2-2437-1319860693-SNAPSHOT' implementation 'com.exactpro.th2:common:3.26.4' implementation 'com.exactpro.th2:sailfish-utils:3.9.1' implementation "org.slf4j:slf4j-log4j12" diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt index 40f38dba..88a5d3e2 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -18,9 +18,9 @@ import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask -import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer import com.exactpro.th2.check1.rule.SailfishFilter +import com.exactpro.th2.check1.rule.preFilterBy import com.exactpro.th2.check1.util.VerificationUtil import com.exactpro.th2.check1.utils.toRootMessageFilter import com.exactpro.th2.common.event.Event @@ -28,7 +28,6 @@ import com.exactpro.th2.common.event.EventUtils.createMessageBean import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.RootMessageFilter -import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toReadableBodyCollection import com.exactpro.th2.common.schema.message.MessageRouter import io.reactivex.Observable @@ -117,20 +116,12 @@ class SilenceCheckTask( } override fun Observable.taskPipeline(): Observable = - map { messageContainer -> // Compare the message with pre-filter - if (LOGGER.isDebugEnabled) { - LOGGER.debug("Pre-filtering message with id: {}", messageContainer.protoMessage.metadata.id.toJson()) - } - val result = matchFilter(messageContainer, messagePreFilter, metadataPreFilter, matchNames = false, significant = false) - ComparisonContainer(messageContainer, protoPreMessageFilter, result) - }.filter { preFilterContainer -> // Filter check result of pre-filter - preFilterContainer.fullyMatches - }.doOnNext { preFilterContainer -> // Update pre-filter state + preFilterBy(this, protoPreMessageFilter, messagePreFilter, metadataPreFilter, LOGGER) { preFilterContainer -> // Update pre-filter state with(preFilterContainer) { preFilterEvent.appendEventsWithVerification(preFilterContainer) preFilterEvent.messageID(protoActual.metadata.id) } - }.map(ComparisonContainer::messageContainer) + } override fun onNext(container: MessageContainer) { container.protoMessage.metadata.apply { From 7dac8b0b1397c9f60819e7f7bd55f99f8dcb567a Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 15 Oct 2021 10:50:34 +0300 Subject: [PATCH 24/41] [TH2-2437] Use dev build for grpc-check1 --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 7e402ae1..d34171e5 100644 --- a/build.gradle +++ b/build.gradle @@ -166,7 +166,7 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0-th2-2437-1319860693-SNAPSHOT' + implementation 'com.exactpro.th2:grpc-check1:3.5.0-dev-1345056567-SNAPSHOT' implementation 'com.exactpro.th2:common:3.26.4' implementation 'com.exactpro.th2:sailfish-utils:3.9.1' implementation "org.slf4j:slf4j-log4j12" From f879b5a9453b5b0318f9a678ab91f86cedf10867 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 15 Oct 2021 12:29:17 +0300 Subject: [PATCH 25/41] [TH2-2437] Introduce object parameter for previous rule data --- .../th2/check1/rule/AbstractCheckTask.kt | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 02ea03d7..2fac9975 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -184,7 +184,7 @@ abstract class AbstractCheckTask( legacy.executorService } legacy.sequenceData.apply { - checkTask.begin(lastSequence, lastMessageTimestamp, executor, untrusted, completed) + checkTask.begin(lastSequence, lastMessageTimestamp, executor, PreviousExecutionData(untrusted, completed)) } } LOGGER.info("Task {} ({}) subscribed to task {} ({})", checkTask.description, checkTask.hashCode(), description, hashCode()) @@ -274,11 +274,10 @@ abstract class AbstractCheckTask( sequence: Long = DEFAULT_SEQUENCE, checkpointTimestamp: Timestamp? = null, executorService: ExecutorService = createExecutorService(), - untrusted: Boolean = false, - parentTaskCompleted: Boolean? = null + previousExecutionData: PreviousExecutionData = PreviousExecutionData.DEFAULT ) { configureRootEvent() - isParentCompleted = parentTaskCompleted + isParentCompleted = previousExecutionData.completed if (!taskState.compareAndSet(State.CREATED, State.BEGIN)) { throw IllegalStateException("Task $description already has been started") } @@ -286,7 +285,7 @@ abstract class AbstractCheckTask( RuleMetric.incrementActiveRule(type()) this.lastSequence = sequence this.executorService = executorService - this.untrusted = untrusted + this.untrusted = previousExecutionData.untrusted this.checkpointTimeout = calculateCheckpointTimeout(checkpointTimestamp, taskTimeout.messageTimeout) this.isDefaultSequence = sequence == DEFAULT_SEQUENCE val scheduler = Schedulers.from(executorService) @@ -708,4 +707,20 @@ abstract class AbstractCheckTask( private data class Legacy(val executorService: ExecutorService, val sequenceData: SequenceData) private data class SequenceData(val lastSequence: Long, val lastMessageTimestamp: Timestamp?, val untrusted: Boolean) + private data class PreviousExecutionData( + /** + * `true` if the previous rule in the chain marked as untrusted + */ + val untrusted: Boolean = false, + /** + * `true` if previous rule has been completed normally. Otherwise, `false` + * + * `null` if there is no previous rule in chain + */ + val completed: Boolean? = null + ) { + companion object { + val DEFAULT = PreviousExecutionData() + } + } } From d1546a09d609cfe426a24cf217d97e4b5788419e Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Fri, 22 Oct 2021 12:09:11 +0400 Subject: [PATCH 26/41] [TH2-114] Added supporting for a new filter operations (#80) * [TH2-114] Added supporting for a new filter operations * [TH2-114] Small code improvements * [TH2-114] Updated sailfish-utils release version * [TH2-114] Added description for a new configuration parameters * [TH2-114] Added link with java duration description --- README.md | 23 +++++++- build.gradle | 6 +- .../configuration/Check1Configuration.java | 23 +++++++- .../th2/check1/entities/RuleConfiguration.kt | 37 ++++++++++++ .../th2/check1/rule/AbstractCheckTask.kt | 38 +++++++----- .../exactpro/th2/check1/rule/RuleFactory.kt | 55 +++++++++++++----- .../th2/check1/rule/check/CheckRuleTask.kt | 8 +-- .../rule/nomessage/NoMessageCheckTask.kt | 8 +-- .../rule/sequence/SequenceCheckRuleTask.kt | 8 +-- .../check1/rule/sequence/SilenceCheckTask.kt | 16 +---- .../check1/entities/RuleConfigurationTest.kt | 58 +++++++++++++++++++ .../th2/check1/rule/AbstractCheckTaskTest.kt | 8 +++ .../com/exactpro/th2/check1/rule/TestChain.kt | 17 ++---- .../check1/rule/check/TestCheckRuleTask.kt | 4 +- .../rule/nomessage/TestNoMessageCheckTask.kt | 7 +-- .../rule/sequence/TestSequenceCheckTask.kt | 5 +- 16 files changed, 232 insertions(+), 89 deletions(-) create mode 100644 src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt create mode 100644 src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt diff --git a/README.md b/README.md index 39a7102c..be00b887 100644 --- a/README.md +++ b/README.md @@ -40,7 +40,10 @@ spec: cleanup-older-than: '60' cleanup-time-unit: 'SECONDS' max-event-batch-content-size: '1048576' + rule-execution-timeout: '5000' auto-silence-check-after-sequence-rule: false + time-precision: 'PT0.000000001S' + decimal-precision: '0.00001' type: th2-check1 pins: - name: server @@ -75,7 +78,9 @@ This block describes the configuration for check1. "cleanup-time-unit": "SECONDS", "max-event-batch-content-size": "1048576", "rule-execution-timeout": 5000, - "auto-silence-check-after-sequence-rule": false + "auto-silence-check-after-sequence-rule": false, + "time-precision": "PT0.000000001S", + "decimal-precision": 0.00001 } ``` @@ -94,9 +99,18 @@ The time unit for _cleanup-older-than_ setting. The available values are MILLIS, #### max-event-batch-content-size The max size in bytes of summary events content in a batch defined in _max-event-batch-content-size_ setting. _The default value is set to 1048576_ +#### rule-execution-timeout +The default rule execution timeout is used if no rule timeout is specified. Measured in milliseconds + #### auto-silence-check-after-sequence-rule Defines a default behavior for creating CheckSequenceRule if `silence_check` parameter is not specified in the request. The default value is `false` +#### time-precision +The time precision is used to compare two time values. It is based on the `ISO-8601` duration format `PnDTnHnMn.nS` with days considered to be exactly 24 hours. Additional information can be found [here](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/time/Duration.html#parse(java.lang.CharSequence)) + +#### decimal-precision +The decimal precision is used to compare two number values. Can be specified in number or string format. For example `0.0001`, `0.125`, `125E-3` + ## Required pins The Check1 component has two types of pin: @@ -139,6 +153,13 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + New configuration parameter `rule-execution-timeout` witch is used if the user has not specified a timeout for the rule execution + Auto silence check after the CheckSequenceRule. + `auto-silence-check-after-sequence-rule` to setup a default behavior for CheckSequenceRule ++ New configuration parameter `time-precision` witch is uses if the user has not specified a time precision ++ New configuration parameter `decimal-precision` witch is uses if the user has not specified a number precision + +#### Changed: ++ Migrated `common` version from `3.26.4` to `3.26.5` ++ Migrated `grpc-check1` version from `3.4.2` to `3.5.0` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.10.0` ### 3.8.0 diff --git a/build.gradle b/build.gradle index d34171e5..4eb772ea 100644 --- a/build.gradle +++ b/build.gradle @@ -166,9 +166,9 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0-dev-1345056567-SNAPSHOT' - implementation 'com.exactpro.th2:common:3.26.4' - implementation 'com.exactpro.th2:sailfish-utils:3.9.1' + implementation 'com.exactpro.th2:grpc-check1:3.5.0-dev-1358229725-SNAPSHOT' + implementation 'com.exactpro.th2:common:3.26.5' + implementation 'com.exactpro.th2:sailfish-utils:3.10.0' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index 0841df51..3ff70b54 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -13,10 +13,13 @@ package com.exactpro.th2.check1.configuration; -import java.time.temporal.ChronoUnit; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; + +import java.time.Duration; +import java.time.temporal.ChronoUnit; public class Check1Configuration { @@ -39,6 +42,14 @@ public class Check1Configuration { @JsonPropertyDescription("The default behavior in case the SequenceCheckRule does not have silenceCheck parameter specified") private boolean autoSilenceCheckAfterSequenceRule; + @JsonProperty(value="decimal-precision", defaultValue = "0.00001") + private double decimalPrecision = 0.00001; + + @JsonProperty(value="time-precision", defaultValue = "PT0.000000001S") + @JsonDeserialize(using = DurationDeserializer.class) + @JsonPropertyDescription("The default time precision value uses java duration format") + private Duration timePrecision = Duration.parse("PT0.000000001S"); + public int getMessageCacheSize() { return messageCacheSize; } @@ -62,4 +73,12 @@ public long getRuleExecutionTimeout() { public boolean isAutoSilenceCheckAfterSequenceRule() { return autoSilenceCheckAfterSequenceRule; } + + public double getDecimalPrecision() { + return decimalPrecision; + } + + public Duration getTimePrecision() { + return timePrecision; + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt new file mode 100644 index 00000000..451ff93b --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt @@ -0,0 +1,37 @@ +/* + * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.entities + +import java.time.Duration + +data class RuleConfiguration( + val taskTimeout: TaskTimeout, + val description: String?, + val timePrecision: Duration, + val decimalPrecision: Double, + val maxEventBatchContentSize: Int +) { + init { + require(!timePrecision.isNegative) { "Time precision cannot be negative" } + require(decimalPrecision >= 0) { "Decimal precision cannot be negative" } + require(maxEventBatchContentSize > 0) { + "'maxEventBatchContentSize' should be greater than zero, actual: $maxEventBatchContentSize" + } + with(taskTimeout) { + require(timeout > 0) { + "'timeout' should be set or be greater than zero, actual: $timeout" + } + } + } +} diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 2fac9975..18113c6d 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -22,6 +22,7 @@ import com.exactpro.th2.check1.AbstractSessionObserver import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.CheckpointData +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.event.bean.builder.VerificationBuilder import com.exactpro.th2.check1.exception.RuleInternalException @@ -42,10 +43,11 @@ import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.MessageMetadata import com.exactpro.th2.common.grpc.MetadataFilter import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.message.toJavaDuration import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.message.toReadableBodyCollection -import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.schema.message.MessageRouter +import com.exactpro.th2.sailfish.utils.FilterSettings import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter import com.google.protobuf.TextFormat.shortDebugString import com.google.protobuf.Timestamp @@ -70,9 +72,7 @@ import java.util.concurrent.atomic.AtomicReference * **Class in not thread-safe** */ abstract class AbstractCheckTask( - val description: String?, - private val taskTimeout: TaskTimeout, - private val maxEventBatchContentSize: Int, + private val ruleConfiguration: RuleConfiguration, submitTime: Instant, protected val sessionKey: SessionKey, private val parentEventID: EventID, @@ -80,14 +80,8 @@ abstract class AbstractCheckTask( private val eventBatchRouter: MessageRouter ) : AbstractSessionObserver() { - init { - require(maxEventBatchContentSize > 0) { - "'maxEventBatchContentSize' should be greater than zero, actual: $maxEventBatchContentSize" - } - require(taskTimeout.timeout > 0) { - "'timeout' should be set or be greater than zero, actual: ${taskTimeout.timeout}" - } - } + val description: String? = ruleConfiguration.description + private val taskTimeout: TaskTimeout = ruleConfiguration.taskTimeout protected var handledMessageCounter: Long = 0 @@ -413,7 +407,7 @@ abstract class AbstractCheckTask( LOGGER.info("Skip event publication for task ${type()} '$description' (${hashCode()})") return } - val batches = rootEvent.disperseToBatches(maxEventBatchContentSize, parentEventID) + val batches = rootEvent.disperseToBatches(ruleConfiguration.maxEventBatchContentSize, parentEventID) RESPONSE_EXECUTOR.execute { batches.forEach { batch -> @@ -543,7 +537,6 @@ abstract class AbstractCheckTask( companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE - const val DEFAULT_TASK_TIMEOUT = 3000L private val RESPONSE_EXECUTOR = ForkJoinPool.commonPool() } @@ -636,7 +629,22 @@ abstract class AbstractCheckTask( protected fun ProtoToIMessageConverter.fromProtoPreFilter(protoPreMessageFilter: RootMessageFilter, messageName: String = protoPreMessageFilter.messageType): IMessage { - return fromProtoFilter(protoPreMessageFilter.messageFilter, messageName) + val filterSettings = protoPreMessageFilter.comparisonSettings.run { + FilterSettings().apply { + decimalPrecision = if (this@run.decimalPrecision.isBlank()) { + ruleConfiguration.decimalPrecision + } else { + this@run.decimalPrecision.toDouble() + } + timePrecision = if (this@run.hasTimePrecision()) { + this@run.timePrecision.toJavaDuration() + } else { + ruleConfiguration.timePrecision + } + } + } + + return fromProtoFilter(protoPreMessageFilter.messageFilter, filterSettings, messageName) } private fun Observable.mapToMessageContainer(): Observable = diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt index 24e9473e..65a6f1a4 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt @@ -16,13 +16,13 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.configuration.Check1Configuration +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.exception.RuleCreationException import com.exactpro.th2.check1.exception.RuleInternalException import com.exactpro.th2.check1.grpc.CheckRuleRequest import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest import com.exactpro.th2.check1.grpc.NoMessageCheckRequest -import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.check.CheckRuleTask import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask @@ -38,7 +38,6 @@ import com.exactpro.th2.common.grpc.RootComparisonSettings import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.message.toJson import com.exactpro.th2.common.schema.message.MessageRouter -import com.google.protobuf.GeneratedMessageV3 import io.reactivex.Observable import mu.KotlinLogging import org.slf4j.Logger @@ -52,6 +51,8 @@ class RuleFactory( ) { private val maxEventBatchContentSize = configuration.maxEventBatchContentSize private val defaultRuleExecutionTimeout = configuration.ruleExecutionTimeout + private val timePrecision = configuration.timePrecision + private val decimalPrecision = configuration.decimalPrecision fun createCheckRule(request: CheckRuleRequest): CheckRuleTask = ruleCreation(request.parentEventId) { @@ -70,14 +71,19 @@ class RuleFactory( } else { request.filter.toRootMessageFilter() }.also { it.validateRootMessageFilter() } - val direction = directionOrDefault(request.direction) - CheckRuleTask( + val ruleConfiguration = RuleConfiguration( + createTaskTimeout(request.timeout, request.messageTimeout), request.description, + timePrecision, + decimalPrecision, + maxEventBatchContentSize + ) + + CheckRuleTask( + ruleConfiguration, Instant.now(), sessionKey, - createTaskTimeout(request.timeout, request.messageTimeout), - maxEventBatchContentSize, filter, request.parentEventId, streamObservable, @@ -109,12 +115,18 @@ class RuleFactory( request.messageFiltersList.map { it.toRootMessageFilter() } }.onEach { it.validateRootMessageFilter() } - SequenceCheckRuleTask( + val ruleConfiguration = RuleConfiguration( + createTaskTimeout(request.timeout, request.messageTimeout), request.description, + timePrecision, + decimalPrecision, + maxEventBatchContentSize + ) + + SequenceCheckRuleTask( + ruleConfiguration, Instant.now(), sessionKey, - createTaskTimeout(request.timeout, request.messageTimeout), - maxEventBatchContentSize, request.preFilter, protoMessageFilters, request.checkOrder, @@ -140,12 +152,18 @@ class RuleFactory( val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } - NoMessageCheckTask( + val ruleConfiguration = RuleConfiguration( + createTaskTimeout(request.timeout, request.messageTimeout), request.description, + timePrecision, + decimalPrecision, + maxEventBatchContentSize + ) + + NoMessageCheckTask( + ruleConfiguration, Instant.now(), sessionKey, - createTaskTimeout(request.timeout, request.messageTimeout), - maxEventBatchContentSize, request.preFilter, parentEventID, streamObservable, @@ -168,11 +186,18 @@ class RuleFactory( check(timeout > 0) { "timeout must be greater that zero" } val sessionAlias: String = request.connectivityId.sessionAlias val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) + + val ruleConfiguration = RuleConfiguration( + createTaskTimeout(timeout), + request.description.takeIf(String::isNotEmpty), + timePrecision, + decimalPrecision, + maxEventBatchContentSize + ) + SilenceCheckTask( + ruleConfiguration, request.preFilter, - request.description.takeIf(String::isNotEmpty), - createTaskTimeout(timeout), - maxEventBatchContentSize, Instant.now(), sessionKey, request.parentEventId, diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index 2211c004..1ae52aca 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -15,7 +15,7 @@ package com.exactpro.th2.check1.rule.check import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer -import com.exactpro.th2.check1.entities.TaskTimeout +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.ComparisonContainer import com.exactpro.th2.check1.rule.MessageContainer @@ -36,16 +36,14 @@ import java.time.Instant * This rule checks for the presence of a single message in the messages stream. */ class CheckRuleTask( - description: String?, + ruleConfiguration: RuleConfiguration, startTime: Instant, sessionKey: SessionKey, - taskTimeout: TaskTimeout, - maxEventBatchContentSize: Int, private val protoMessageFilter: RootMessageFilter, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(ruleConfiguration, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val messageFilter: SailfishFilter = SailfishFilter( converter.fromProtoPreFilter(protoMessageFilter), diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 25d619a2..3dacd01e 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -15,7 +15,7 @@ package com.exactpro.th2.check1.rule.nomessage import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer -import com.exactpro.th2.check1.entities.TaskTimeout +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.MessageContainer @@ -34,16 +34,14 @@ import io.reactivex.Observable import java.time.Instant class NoMessageCheckTask( - description: String?, + ruleConfiguration: RuleConfiguration, startTime: Instant, sessionKey: SessionKey, - taskTimeout: TaskTimeout, - maxEventBatchContentSize: Int, protoPreFilter: PreFilter, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(ruleConfiguration, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index 7ece4bb8..af10c5fa 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -17,7 +17,7 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer -import com.exactpro.th2.check1.entities.TaskTimeout +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.event.CheckSequenceUtils import com.exactpro.th2.check1.event.bean.CheckSequenceRow import com.exactpro.th2.check1.grpc.PreFilter @@ -58,18 +58,16 @@ import kotlin.collections.set * If this parameter is set to `false`, the order won't be checked. */ class SequenceCheckRuleTask( - description: String?, + ruleConfiguration: RuleConfiguration, startTime: Instant, sessionKey: SessionKey, - taskTimeout: TaskTimeout, - maxEventBatchContentSize: Int, protoPreFilter: PreFilter, private val protoMessageFilters: List, private val checkOrder: Boolean, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask(description, taskTimeout, maxEventBatchContentSize, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { +) : AbstractCheckTask(ruleConfiguration, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt index 88a5d3e2..7fc3b478 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -15,6 +15,7 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask @@ -35,25 +36,14 @@ import java.time.Instant import java.util.concurrent.atomic.AtomicBoolean class SilenceCheckTask( + ruleConfiguration: RuleConfiguration, protoPreFilter: PreFilter, - description: String?, - taskTimeout: TaskTimeout, - maxEventBatchContentSize: Int, submitTime: Instant, sessionKey: SessionKey, parentEventID: EventID, messageStream: Observable, eventBatchRouter: MessageRouter -) : AbstractCheckTask( - description, - taskTimeout, - maxEventBatchContentSize, - submitTime, - sessionKey, - parentEventID, - messageStream, - eventBatchRouter -) { +) : AbstractCheckTask(ruleConfiguration, submitTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( converter.fromProtoPreFilter(protoPreMessageFilter), diff --git a/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt b/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt new file mode 100644 index 00000000..9502d4ed --- /dev/null +++ b/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt @@ -0,0 +1,58 @@ +/* + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.entities + +import org.junit.jupiter.api.Test +import org.junit.jupiter.api.assertThrows +import java.time.Duration +import kotlin.test.assertEquals + +class RuleConfigurationTest { + + @Test + fun `check that time precision is negative`() { + val exception = assertThrows { + RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(-1), 0.005, 1) + } + assertEquals(exception.message, "Time precision cannot be negative") + } + + @Test + fun `check that decimal precision is negative`() { + val exception = assertThrows { + RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(1), -0.005, 1) + } + assertEquals(exception.message, "Decimal precision cannot be negative") + } + + @Test + fun `check that max event batch content size is negative`() { + val maxEventBatchContentSize = -1 + + val exception = assertThrows { + RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(1), 0.005, maxEventBatchContentSize) + } + assertEquals(exception.message, "'maxEventBatchContentSize' should be greater than zero, actual: $maxEventBatchContentSize") + } + + @Test + fun `check that task timeout is negative`() { + val timeout = -1L + + val exception = assertThrows { + RuleConfiguration(TaskTimeout(timeout, 0), null, Duration.ofSeconds(1), 0.005, 1) + } + assertEquals(exception.message, "'timeout' should be set or be greater than zero, actual: $timeout") + } +} \ No newline at end of file diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index 695a7b7f..83a78979 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -14,6 +14,9 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer +import com.exactpro.th2.check1.configuration.Check1Configuration +import com.exactpro.th2.check1.entities.RuleConfiguration +import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.common.event.EventUtils import com.exactpro.th2.common.event.IBodyData @@ -46,6 +49,7 @@ import kotlin.test.assertNotNull abstract class AbstractCheckTaskTest { protected val clientStub: MessageRouter = spy { } + protected val configuration = Check1Configuration() fun awaitEventBatchRequest(timeoutValue: Long = 1000L, times: Int): List { val argumentCaptor = argumentCaptor() @@ -152,6 +156,10 @@ abstract class AbstractCheckTaskTest { } } + protected fun createRuleConfiguration(taskTimeout: TaskTimeout, description: String = "Test", maxEventBatchContentSize: Int = 1024 * 1024): RuleConfiguration { + return RuleConfiguration(taskTimeout, description, configuration.timePrecision, configuration.decimalPrecision, maxEventBatchContentSize) + } + @JsonSubTypes(value = [ JsonSubTypes.Type(value = Verification::class, name = Verification.TYPE), diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt index c8d11ba4..a8d83f7f 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/TestChain.kt @@ -273,15 +273,12 @@ class TestChain: AbstractCheckTaskTest() { messageStream: Observable, checkOrder: Boolean = true, preFilterParam: PreFilter = preFilter, - maxEventBatchContentSize: Int = 1024 * 1024, taskTimeout: TaskTimeout = TaskTimeout(1000L) ): SequenceCheckRuleTask { return SequenceCheckRuleTask( - description = "Test", + ruleConfiguration = createRuleConfiguration(taskTimeout), startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, FIRST), - taskTimeout = taskTimeout, - maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = sequence.map(::createMessageFilter).toList(), checkOrder = checkOrder, @@ -295,14 +292,11 @@ class TestChain: AbstractCheckTaskTest() { sequence: Int, parentEventID: EventID, messageStream: Observable, - maxEventBatchContentSize: Int = 1024 * 1024, taskTimeout: TaskTimeout = TaskTimeout(1000L) ) = CheckRuleTask( - SESSION_ALIAS, + createRuleConfiguration(taskTimeout, SESSION_ALIAS), Instant.now(), SessionKey(SESSION_ALIAS, FIRST), - taskTimeout, - maxEventBatchContentSize, createMessageFilter(sequence), parentEventID, messageStream, @@ -313,15 +307,12 @@ class TestChain: AbstractCheckTaskTest() { parentEventID: EventID, messageStream: Observable, preFilterParam: PreFilter, - taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L), - maxEventBatchContentSize: Int = 1024 * 1024 + taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L) ): NoMessageCheckTask { return NoMessageCheckTask( - description = "Test", + ruleConfiguration = createRuleConfiguration(taskTimeout), startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, FIRST), - taskTimeout = taskTimeout, - maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, parentEventID = parentEventID, messageStream = messageStream, diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt index 17952dd7..de2b76f8 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/check/TestCheckRuleTask.kt @@ -62,11 +62,9 @@ internal class TestCheckRuleTask : AbstractCheckTaskTest() { maxEventBatchContentSize: Int = 1024 * 1024, taskTimeout: TaskTimeout = TaskTimeout(1000L) ) = CheckRuleTask( - SESSION_ALIAS, + createRuleConfiguration(taskTimeout, SESSION_ALIAS, maxEventBatchContentSize), Instant.now(), SessionKey(SESSION_ALIAS, Direction.FIRST), - taskTimeout, - maxEventBatchContentSize, messageFilter, parentEventID, messageStream, diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt index e1d9ca04..6daaa953 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/nomessage/TestNoMessageCheckTask.kt @@ -195,15 +195,12 @@ class TestNoMessageCheckTask : AbstractCheckTaskTest() { parentEventID: EventID, messageStream: Observable, preFilterParam: PreFilter, - taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L), - maxEventBatchContentSize: Int = 1024 * 1024 + taskTimeout: TaskTimeout = TaskTimeout(5000L, 3500L) ): NoMessageCheckTask { return NoMessageCheckTask( - description = "Test", + ruleConfiguration = createRuleConfiguration(taskTimeout), startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, Direction.FIRST), - taskTimeout = taskTimeout, - maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, parentEventID = parentEventID, messageStream = messageStream, diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt index 7efe99a7..6af402bc 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTask.kt @@ -779,15 +779,12 @@ class TestSequenceCheckTask : AbstractCheckTaskTest() { checkOrder: Boolean, preFilterParam: PreFilter = preFilter, filtersParam: List = protoMessageFilters, - maxEventBatchContentSize: Int = 1024 * 1024, taskTimeout: TaskTimeout = TaskTimeout(5000L) ): SequenceCheckRuleTask { return SequenceCheckRuleTask( - description = "Test", + ruleConfiguration = createRuleConfiguration(taskTimeout), startTime = Instant.now(), sessionKey = SessionKey(SESSION_ALIAS, Direction.FIRST), - taskTimeout = taskTimeout, - maxEventBatchContentSize = maxEventBatchContentSize, protoPreFilter = preFilterParam, protoMessageFilters = filtersParam, checkOrder = checkOrder, From e063aa8aa20f057aa06845ac635895145d429f34 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Tue, 9 Nov 2021 20:07:28 +0400 Subject: [PATCH 27/41] [TH2-2642] Migrated sailfish-utils version to 3.10.1 (#85) * Fixed conversion of null values --- README.md | 3 ++- build.gradle | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index be00b887..d0777440 100644 --- a/README.md +++ b/README.md @@ -159,7 +159,8 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type #### Changed: + Migrated `common` version from `3.26.4` to `3.26.5` + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.10.0` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.10.1` + + Fixed conversion of `null` values ### 3.8.0 diff --git a/build.gradle b/build.gradle index 4eb772ea..49ef4192 100644 --- a/build.gradle +++ b/build.gradle @@ -166,9 +166,9 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0-dev-1358229725-SNAPSHOT' + implementation 'com.exactpro.th2:grpc-check1:3.5.0' implementation 'com.exactpro.th2:common:3.26.5' - implementation 'com.exactpro.th2:sailfish-utils:3.10.0' + implementation 'com.exactpro.th2:sailfish-utils:3.10.1' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" From 47bb4b6f7325557045894c0117ed368fa45f78fa Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Wed, 10 Nov 2021 12:45:00 +0400 Subject: [PATCH 28/41] [TH2-2654] Fixed replacement of the expected result with the string "null" value (#88) * [TH2-2654] Fixed replacement of the expected result with the string "null" value * Update README.md Co-authored-by: Oleg Smirnov --- README.md | 1 + .../check1/event/VerificationEntryUtils.java | 8 ++- .../event/TestVerificationEntryUtils.kt | 67 ++++++++++++++++++- 3 files changed, 73 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index d0777440..7d15d406 100644 --- a/README.md +++ b/README.md @@ -161,6 +161,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` + Migrated `sailfish-utils` version from `3.9.1` to `3.10.1` + Fixed conversion of `null` values ++ Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value ### 3.8.0 diff --git a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java index b6edde0b..141ebabd 100644 --- a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java +++ b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2020 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -34,7 +34,7 @@ public class VerificationEntryUtils { public static VerificationEntry createVerificationEntry(ComparisonResult result) { VerificationEntry verificationEntry = new VerificationEntry(); verificationEntry.setActual(Objects.toString(result.getActual(), null)); - verificationEntry.setExpected(Formatter.formatExpected(result)); + verificationEntry.setExpected(convertExpectedResult(result)); verificationEntry.setStatus(toVerificationStatus(result.getStatus())); verificationEntry.setKey(result.isKey()); verificationEntry.setOperation(resolveOperation(result)); @@ -89,4 +89,8 @@ private static VerificationStatus toVerificationStatus(StatusType statusType) { throw new IllegalArgumentException("Unsupported status type '" + statusType + '\''); } } + + private static String convertExpectedResult(ComparisonResult result) { + return result.getExpected() == null ? null : Formatter.formatExpected(result); + } } diff --git a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt index a9e8157d..8f1d0d9d 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt @@ -1,5 +1,5 @@ /* - * Copyright 2020-2020 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2021 Exactpro (Exactpro Systems Limited) * 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 @@ -23,6 +23,7 @@ import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.message.message import com.exactpro.th2.common.value.toValue +import com.exactpro.th2.common.value.toValueFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter import com.fasterxml.jackson.databind.ObjectMapper import org.junit.jupiter.api.Assertions @@ -188,6 +189,70 @@ class TestVerificationEntryUtils { Assertions.assertTrue(msgEntry.isKey) { "msg is not a key field in ${msgEntry.toDebugString()}" } } + @Test + fun `expected value converted as null value`() { + val filter: RootMessageFilter = RootMessageFilter.newBuilder() + .setMessageType("Test") + .setMessageFilter(MessageFilter.newBuilder() + .putFields("A", "1".toValueFilter()) + .build()) + .build() + + val actual = message("Test").apply { + putFields("A", "1".toValue()) + putFields("B", "2".toValue()) + }.build() + + val container = VerificationUtil.toMetaContainer(filter.messageFilter, false) + val settings = ComparatorSettings().apply { + metaContainer = container + } + + val actualIMessage = converter.fromProtoMessage(actual, false) + val filterIMessage = converter.fromProtoFilter(filter.messageFilter, filter.messageType) + val result = MessageComparator.compare( + actualIMessage, + filterIMessage, + settings + ) + + val entry = VerificationEntryUtils.createVerificationEntry(result) + val keyEntry = entry.fields["B"].assertNotNull { "The key 'B' is missing in ${entry.toDebugString()}" } + Assertions.assertNull(keyEntry.expected) { "Expected value should be null" } + } + + @Test + fun `actual value converted as null value`() { + val filter: RootMessageFilter = RootMessageFilter.newBuilder() + .setMessageType("Test") + .setMessageFilter(MessageFilter.newBuilder() + .putFields("A", "1".toValueFilter()) + .putFields("B", "2".toValueFilter()) + .build()) + .build() + + val actual = message("Test").apply { + putFields("A", "1".toValue()) + }.build() + + val container = VerificationUtil.toMetaContainer(filter.messageFilter, false) + val settings = ComparatorSettings().apply { + metaContainer = container + } + + val actualIMessage = converter.fromProtoMessage(actual, false) + val filterIMessage = converter.fromProtoFilter(filter.messageFilter, filter.messageType) + val result = MessageComparator.compare( + actualIMessage, + filterIMessage, + settings + ) + + val entry = VerificationEntryUtils.createVerificationEntry(result) + val keyEntry = entry.fields["B"].assertNotNull { "The key 'B' is missing in ${entry.toDebugString()}" } + Assertions.assertNull(keyEntry.actual) { "Actual value should be null" } + } + companion object { private fun VerificationEntry.toDebugString(): String = ObjectMapper().writeValueAsString(this) private fun T?.assertNotNull(msg: () -> String): T { From 905725313247f313074da8fa3cd67058f9e95604 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Tue, 23 Nov 2021 12:54:33 +0400 Subject: [PATCH 29/41] [TH2-2644] Added new hint parameter for VerificationEntry (#92) * [TH2-2644] Added new hint parameter for VerificationEntry * [TH2-2644] Updated sailfish-utils version * Covered with tests * [TH2-2644] Corrected README.md --- README.md | 11 ++- build.gradle | 4 +- .../check1/event/VerificationEntryUtils.java | 6 ++ .../th2/check1/metrics/BufferMetric.kt | 6 +- .../event/TestVerificationEntryUtils.kt | 93 +++++++++++++++++++ 5 files changed, 110 insertions(+), 10 deletions(-) diff --git a/README.md b/README.md index 7d15d406..f2941aeb 100644 --- a/README.md +++ b/README.md @@ -150,16 +150,17 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type #### Added: + Implemented NoMessageCheck rule task. Updated CheckRule and CheckSequence rule tasks -+ New configuration parameter `rule-execution-timeout` witch is used if the user has not specified a timeout for the rule execution ++ New configuration parameter `rule-execution-timeout` which is used if the user has not specified a timeout for the rule execution + Auto silence check after the CheckSequenceRule. + `auto-silence-check-after-sequence-rule` to setup a default behavior for CheckSequenceRule -+ New configuration parameter `time-precision` witch is uses if the user has not specified a time precision -+ New configuration parameter `decimal-precision` witch is uses if the user has not specified a number precision ++ New configuration parameter `time-precision` which is uses if the user has not specified a time precision ++ New configuration parameter `decimal-precision` which is uses if the user has not specified a number precision ++ New parameter `hint` for verification event which is used to display the reason for the failed field comparison. For example the type mismatch of the compared values #### Changed: -+ Migrated `common` version from `3.26.4` to `3.26.5` ++ Migrated `common` version from `3.26.4` to `3.28.0` + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.10.1` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.10.2` + Fixed conversion of `null` values + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value diff --git a/build.gradle b/build.gradle index 49ef4192..81a877df 100644 --- a/build.gradle +++ b/build.gradle @@ -167,8 +167,8 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') implementation 'com.exactpro.th2:grpc-check1:3.5.0' - implementation 'com.exactpro.th2:common:3.26.5' - implementation 'com.exactpro.th2:sailfish-utils:3.10.1' + implementation 'com.exactpro.th2:common:3.28.0' + implementation 'com.exactpro.th2:sailfish-utils:3.10.2' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" diff --git a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java index 141ebabd..756914df 100644 --- a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java +++ b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java @@ -38,6 +38,7 @@ public static VerificationEntry createVerificationEntry(ComparisonResult result) verificationEntry.setStatus(toVerificationStatus(result.getStatus())); verificationEntry.setKey(result.isKey()); verificationEntry.setOperation(resolveOperation(result)); + verificationEntry.setHint(extractHint(result)); if (result.hasResults()) { verificationEntry.setFields(result.getResults().entrySet().stream() .collect(Collectors.toMap( @@ -93,4 +94,9 @@ private static VerificationStatus toVerificationStatus(StatusType statusType) { private static String convertExpectedResult(ComparisonResult result) { return result.getExpected() == null ? null : Formatter.formatExpected(result); } + + private static String extractHint(ComparisonResult result) { + Exception exception = result.getException(); + return exception == null ? null : exception.getMessage(); + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/metrics/BufferMetric.kt b/src/main/kotlin/com/exactpro/th2/check1/metrics/BufferMetric.kt index a415547d..6a1daf2f 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/metrics/BufferMetric.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/metrics/BufferMetric.kt @@ -15,8 +15,8 @@ package com.exactpro.th2.check1.metrics import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.configuration.Check1Configuration -import com.exactpro.th2.common.metrics.DEFAULT_DIRECTION_LABEL_NAME -import com.exactpro.th2.common.metrics.DEFAULT_SESSION_ALIAS_LABEL_NAME +import com.exactpro.th2.common.metrics.DIRECTION_LABEL +import com.exactpro.th2.common.metrics.SESSION_ALIAS_LABEL import io.prometheus.client.Counter import java.util.concurrent.ConcurrentHashMap import kotlin.math.min @@ -25,7 +25,7 @@ object BufferMetric { private val actualBufferCountMetric: Counter = Counter .build("th2_check1_actual_cache_number", "The actual number of messages in caches") - .labelNames(DEFAULT_SESSION_ALIAS_LABEL_NAME, DEFAULT_DIRECTION_LABEL_NAME) + .labelNames(SESSION_ALIAS_LABEL, DIRECTION_LABEL) .register() private val bufferMessagesSizeBySessionKey: MutableMap = ConcurrentHashMap() diff --git a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt index 8f1d0d9d..88ecef6d 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt @@ -20,8 +20,10 @@ import com.exactpro.th2.common.event.bean.VerificationEntry import com.exactpro.th2.common.grpc.ListValueFilter import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.grpc.Value import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.message.message +import com.exactpro.th2.common.message.messageFilter import com.exactpro.th2.common.value.toValue import com.exactpro.th2.common.value.toValueFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter @@ -29,6 +31,11 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertAll +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.Arguments.arguments +import org.junit.jupiter.params.provider.MethodSource +import java.util.stream.Stream class TestVerificationEntryUtils { private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null) @@ -253,11 +260,97 @@ class TestVerificationEntryUtils { Assertions.assertNull(keyEntry.actual) { "Actual value should be null" } } + @ParameterizedTest + @MethodSource("unexpectedTypeMismatch") + fun `verify messages with different value type`(actualValue: Value, expectedValueFilter: ValueFilter, expectedHint: String?) { + val filter: RootMessageFilter = RootMessageFilter.newBuilder() + .setMessageType("Test") + .setMessageFilter( + messageFilter().apply { + putFields("A", "1".toValueFilter()) + putFields("B", expectedValueFilter) + }.build()) + .build() + + val actual = message("Test").apply { + putFields("A", "1".toValue()) + putFields("B", actualValue) + }.build() + + val actualIMessage = converter.fromProtoMessage(actual, false) + val filterIMessage = converter.fromProtoFilter(filter.messageFilter, filter.messageType) + val result = MessageComparator.compare( + actualIMessage, + filterIMessage, + ComparatorSettings() + ) + + val entry = VerificationEntryUtils.createVerificationEntry(result) + val keyEntry = entry.fields["B"].assertNotNull { "The key 'B' is missing in ${entry.toDebugString()}" } + Assertions.assertEquals(expectedHint, keyEntry.hint, "Hint must be equal") + } + + companion object { private fun VerificationEntry.toDebugString(): String = ObjectMapper().writeValueAsString(this) private fun T?.assertNotNull(msg: () -> String): T { Assertions.assertNotNull(this, msg) return this!! } + + @JvmStatic + fun unexpectedTypeMismatch(): Stream = Stream.of( + arguments("2".toValue(), "2".toValueFilter(), null), + arguments( + message().putFields("A", "1".toValue()).toValue(), + messageFilter().putFields("A", "1".toValueFilter()).toValueFilter(), + null + ), + arguments( + listOf("2".toValue()).toValue(), + listOf("2".toValueFilter()).toValueFilter(), + null + ), + arguments( + "2".toValue(), + messageFilter().putFields("A", "1".toValueFilter()).toValueFilter(), + "Value type mismatch - actual: String, expected: Message" + ), + arguments( + "2".toValue(), + listOf(messageFilter().putFields("A", "1".toValueFilter())).toValueFilter(), + "Value type mismatch - actual: String, expected: Collection of Messages" + ), + arguments( + message().putFields("A", "1".toValue()).toValue(), + listOf(messageFilter().putFields("A", "1".toValueFilter())).toValueFilter(), + "Value type mismatch - actual: Message, expected: Collection of Messages" + ), + arguments( + "2".toValue(), + listOf("2".toValueFilter()).toValueFilter(), + "Value type mismatch - actual: String, expected: Collection" + ), + arguments( + message().putFields("A", "1".toValue()).toValue(), + "2".toValueFilter(), + "Value type mismatch - actual: Message, expected: String" + ), + arguments( + listOf(message().putFields("A", "1".toValue()).build()).toValue(), + "2".toValueFilter(), + "Value type mismatch - actual: Collection of Messages, expected: String" + ), + arguments( + listOf(message().putFields("A", "1".toValue()).build()).toValue(), + messageFilter().putFields("A", "1".toValueFilter()).toValueFilter(), + "Value type mismatch - actual: Collection of Messages, expected: Message" + ), + arguments( + message().putFields("A", "1".toValue()).toValue(), + listOf("2".toValueFilter()).toValueFilter(), + "Value type mismatch - actual: Message, expected: Collection" + ) + ) } } \ No newline at end of file From ce95860e41d5155998513b9c8329d28977cabe7c Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 25 Nov 2021 12:44:52 +0400 Subject: [PATCH 30/41] [TH2-2736] FailUnexpected is not propagated to children (#101) --- README.md | 1 + .../th2/check1/util/VerificationUtil.java | 16 +-- .../th2/check1/util/TestVerificationUtil.kt | 104 ++++++++++++++++++ 3 files changed, 113 insertions(+), 8 deletions(-) diff --git a/README.md b/README.md index f2941aeb..c6b44b79 100644 --- a/README.md +++ b/README.md @@ -183,6 +183,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated sailfish-utils from `3.7.0` to `3.8.1` + Now Check1 keep the order of repeating result groups by default + Fix IN, NOT_IN FilterOperation interaction ++ Fixed setting of the `failUnexpected` parameter while converting a message filter ### 3.7.2 diff --git a/src/main/java/com/exactpro/th2/check1/util/VerificationUtil.java b/src/main/java/com/exactpro/th2/check1/util/VerificationUtil.java index 7213ba6b..a7133e93 100644 --- a/src/main/java/com/exactpro/th2/check1/util/VerificationUtil.java +++ b/src/main/java/com/exactpro/th2/check1/util/VerificationUtil.java @@ -58,11 +58,6 @@ public static MetaContainer toMetaContainer(MetadataFilter metadataFilter) { public static MetaContainer toMetaContainer(MessageFilter messageFilter, boolean listItemAsSeparate) { MetaContainer metaContainer = new MetaContainer(); - Set keyFields = new HashSet<>(); - - messageFilter.getFieldsMap().forEach((name, value) -> { - toMetaContainer(name, value, metaContainer, keyFields, listItemAsSeparate); - }); if (messageFilter.hasComparisonSettings()) { FailUnexpected failUnexpected = messageFilter.getComparisonSettings().getFailUnexpected(); @@ -73,6 +68,13 @@ public static MetaContainer toMetaContainer(MessageFilter messageFilter, boolean metaContainer.setFailUnexpected(AMLLangConst.ALL); } } + + Set keyFields = new HashSet<>(); + + messageFilter.getFieldsMap().forEach((name, value) -> { + toMetaContainer(name, value, metaContainer, keyFields, listItemAsSeparate); + }); + metaContainer.setKeyFields(keyFields); @@ -96,13 +98,11 @@ private static void toMetaContainer(String fieldName, ValueFilter value, MetaCon } private static void convertList(MetaContainer parent, String fieldName, ListValueFilter listFilter) { - List result = new ArrayList<>(); for (ValueFilter valueFilter : listFilter.getValuesList()) { if (valueFilter.hasMessageFilter()) { - result.add(toMetaContainer(valueFilter.getMessageFilter(), false)); + parent.add(fieldName, toMetaContainer(valueFilter.getMessageFilter(), false)); } } - parent.getChildren().put(fieldName, result); } private static void convertListAsSeparateContainers(MetaContainer parent, String fieldName, diff --git a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt index acb065de..90db7b1f 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt @@ -13,10 +13,29 @@ package com.exactpro.th2.check1.util +import com.exactpro.sf.comparison.ComparatorSettings +import com.exactpro.sf.comparison.ComparisonUtil +import com.exactpro.sf.comparison.MessageComparator +import com.exactpro.sf.scriptrunner.StatusType +import com.exactpro.th2.common.grpc.ComparisonSettings +import com.exactpro.th2.common.grpc.FailUnexpected import com.exactpro.th2.common.grpc.FilterOperation +import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.MetadataFilter +import com.exactpro.th2.common.grpc.RootMessageFilter +import com.exactpro.th2.common.grpc.Value +import com.exactpro.th2.common.grpc.ValueFilter +import com.exactpro.th2.common.message.message +import com.exactpro.th2.common.message.messageFilter +import com.exactpro.th2.common.value.toValue +import com.exactpro.th2.common.value.toValueFilter +import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test +import org.junit.jupiter.params.ParameterizedTest +import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.MethodSource +import java.util.stream.Stream internal class TestVerificationUtil { @Test @@ -33,4 +52,89 @@ internal class TestVerificationUtil { "keyProp" to false /*not transitive*/ ), metaContainer.keyFields) } + + @ParameterizedTest + @MethodSource("failUnexpectedByFieldsAndMessages") + fun `fail unexpected test`(valueFilter: ValueFilter, value: Value) { + val filter: RootMessageFilter = RootMessageFilter.newBuilder() + .setMessageType("Test") + .setMessageFilter(MessageFilter.newBuilder() + .putFields("A", valueFilter) + .setComparisonSettings(ComparisonSettings.newBuilder() + .setFailUnexpected(FailUnexpected.FIELDS_AND_MESSAGES) + .build() + ) + .build()) + .build() + + val actual = message("Test").apply { + putFields("A", value) + }.build() + + val settings = ComparatorSettings().apply { + metaContainer = VerificationUtil.toMetaContainer(filter.messageFilter, false) + } + + val actualIMessage = converter.fromProtoMessage(actual, false) + val filterIMessage = converter.fromProtoFilter(filter.messageFilter, filter.messageType) + val result = MessageComparator.compare( + actualIMessage, + filterIMessage, + settings + ) + + Assertions.assertNotNull(result) { "Result cannot be null" } + Assertions.assertEquals(StatusType.FAILED, ComparisonUtil.getStatusType(result)) + } + + companion object { + private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null) + + @JvmStatic + fun failUnexpectedByFieldsAndMessages(): Stream = Stream.of( + Arguments.arguments( + listOf( + messageFilter().putFields("A1", "1".toValueFilter()) + ).toValueFilter(), + listOf( + message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build() + ).toValue() + ), + Arguments.arguments( + messageFilter().putFields("A1", "1".toValueFilter()).toValueFilter(), + message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build().toValue() + ), + Arguments.arguments( + messageFilter().putFields( + "A1", listOf("1", "2").toValueFilter() + ).toValueFilter(), + message().putFields( + "A1", + listOf("1", "2", "3").toValue() + ).build().toValue() + ), + Arguments.arguments( + messageFilter().putFields( + "A1", + messageFilter().putFields("A2", "1".toValueFilter()).toValueFilter() + ).toValueFilter(), + message().putFields( + "A1", + message() + .putFields("A2", "1".toValue()) + .putFields("B2", "2".toValue()) + .build().toValue() + ).build().toValue() + ), + Arguments.arguments( + listOf( + "1".toValueFilter() + ).toValueFilter(), + listOf( + "1".toValue(), + "2".toValue() + ).toValue() + ) + ) + } } \ No newline at end of file From 0404c5fbec56ae5b936488331b6476b2c8176322 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Thu, 25 Nov 2021 19:13:14 +0300 Subject: [PATCH 31/41] [TH2-2739] Update sailfish-core version. It contains fixes for incorrect matching inside repeating group when the elements are reordered and the filter has less elements than the actual message --- README.md | 2 ++ build.gradle | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index c6b44b79..6f581b68 100644 --- a/README.md +++ b/README.md @@ -163,6 +163,8 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated `sailfish-utils` version from `3.9.1` to `3.10.2` + Fixed conversion of `null` values + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value ++ Migrated `sailfish-core` version to `3.2.1752` + + Fix incorrect matching in repeating groups with reordered messages ### 3.8.0 diff --git a/build.gradle b/build.gradle index 81a877df..e982a123 100644 --- a/build.gradle +++ b/build.gradle @@ -10,7 +10,7 @@ plugins { ext { sharedDir = file("${project.rootDir}/shared") - sailfishVersion = '3.2.1050' + sailfishVersion = '3.2.1752' } group = 'com.exactpro.th2' From 5bab67d06cca603a8460d5e1df264fea5013d8b7 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Fri, 26 Nov 2021 18:35:42 +0400 Subject: [PATCH 32/41] [TH2-2458] Improvement checkpoint validation (#100) * [TH2-2458] Improvement checkpoint validation * [TH2-2458] Small refactoring * [TH2-2458] Covered with tests --- README.md | 2 +- .../exactpro/th2/check1/CollectorService.kt | 18 +- .../th2/check1/entities/RequestAdaptor.kt | 52 +++ .../exactpro/th2/check1/rule/RuleFactory.kt | 34 +- .../th2/check1/rule/RuleFactoryTest.kt | 361 +++++++++++++++++- .../TestSequenceCheckTaskWithSilenceCheck.kt | 10 +- .../com/exactpro/th2/check1/util/Utils.kt | 13 + 7 files changed, 460 insertions(+), 30 deletions(-) create mode 100644 src/main/kotlin/com/exactpro/th2/check1/entities/RequestAdaptor.kt diff --git a/README.md b/README.md index 6f581b68..d36409cb 100644 --- a/README.md +++ b/README.md @@ -163,6 +163,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated `sailfish-utils` version from `3.9.1` to `3.10.2` + Fixed conversion of `null` values + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value ++ Fixed setting of the `failUnexpected` parameter while converting a message filter + Migrated `sailfish-core` version to `3.2.1752` + Fix incorrect matching in repeating groups with reordered messages @@ -185,7 +186,6 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated sailfish-utils from `3.7.0` to `3.8.1` + Now Check1 keep the order of repeating result groups by default + Fix IN, NOT_IN FilterOperation interaction -+ Fixed setting of the `failUnexpected` parameter while converting a message filter ### 3.7.2 diff --git a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt index 95d59fa6..52766748 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/CollectorService.kt @@ -90,11 +90,11 @@ class CollectorService( @Throws(InterruptedException::class) fun verifyCheckRule(request: CheckRuleRequest): ChainID { val chainID = request.getChainIdOrGenerate() - val task = ruleFactory.createCheckRule(request) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) eventIdToLastCheckTask.compute(CheckTaskKey(chainID, request.connectivityId)) { _, value -> + val task = ruleFactory.createCheckRule(request, value != null) task.apply { addToChainOrBegin(value, request.checkpoint) } } return chainID @@ -103,7 +103,6 @@ class CollectorService( @Throws(InterruptedException::class) fun verifyCheckSequenceRule(request: CheckSequenceRuleRequest): ChainID { val chainID = request.getChainIdOrGenerate() - val task = ruleFactory.createSequenceCheckRule(request) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) val silenceCheck = if (request.hasSilenceCheck()) request.silenceCheck.value else defaultAutoSilenceCheck @@ -115,6 +114,7 @@ class CollectorService( } eventIdToLastCheckTask.compute(CheckTaskKey(chainID, request.connectivityId)) { _, value -> + val task = ruleFactory.createSequenceCheckRule(request, value != null) task.apply { addToChainOrBegin(value, request.checkpoint) } .run { silenceCheckTask?.also { subscribeNextTask(it) } ?: this } } @@ -123,20 +123,24 @@ class CollectorService( fun verifyNoMessageCheck(request: NoMessageCheckRequest): ChainID { val chainID = request.getChainIdOrGenerate() - val task = ruleFactory.createNoMessageCheckRule(request) cleanupTasksOlderThan(olderThanDelta, olderThanTimeUnit) eventIdToLastCheckTask.compute(CheckTaskKey(chainID, request.connectivityId)) { _, value -> + val task = ruleFactory.createNoMessageCheckRule(request, value != null) task.apply { addToChainOrBegin(value, request.checkpoint) } } return chainID } - private fun AbstractCheckTask.addToChainOrBegin( - value: AbstractCheckTask?, - checkpoint: GrpcCheckpoint - ): Unit = value?.subscribeNextTask(this) ?: begin(checkpoint) + private fun AbstractCheckTask.addToChainOrBegin(value: AbstractCheckTask?, checkpoint: GrpcCheckpoint) { + val realCheckpoint = if (checkpoint === GrpcCheckpoint.getDefaultInstance()) { + null + } else { + checkpoint + } + value?.subscribeNextTask(this) ?: begin(realCheckpoint) + } private fun CheckRuleRequest.getChainIdOrGenerate(): ChainID { return if (hasChainId()) { diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/RequestAdaptor.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/RequestAdaptor.kt new file mode 100644 index 00000000..750b3fa2 --- /dev/null +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/RequestAdaptor.kt @@ -0,0 +1,52 @@ +/* + * Copyright 2021 Exactpro (Exactpro Systems Limited) + * 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. + */ + +package com.exactpro.th2.check1.entities + +import com.exactpro.th2.check1.grpc.ChainID +import com.exactpro.th2.check1.grpc.CheckRuleRequest +import com.exactpro.th2.check1.grpc.CheckSequenceRuleRequest +import com.exactpro.th2.check1.grpc.NoMessageCheckRequest +import com.exactpro.th2.common.grpc.Checkpoint + +class RequestAdaptor(val chainId: ChainID?, val checkpoint: Checkpoint?) { + + companion object { + fun from(request: CheckRuleRequest): RequestAdaptor { + return request.run { + RequestAdaptor( + if (hasChainId()) chainId else null, + if (hasCheckpoint()) checkpoint else null + ) + } + } + + fun from(request: CheckSequenceRuleRequest): RequestAdaptor { + return request.run { + RequestAdaptor( + if (hasChainId()) chainId else null, + if (hasCheckpoint()) checkpoint else null + ) + } + } + + fun from(request: NoMessageCheckRequest): RequestAdaptor { + return request.run { + RequestAdaptor( + if (hasChainId()) chainId else null, + if (hasCheckpoint()) checkpoint else null + ) + } + } + } +} \ No newline at end of file diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt index 65a6f1a4..3e5037fb 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt @@ -16,6 +16,7 @@ package com.exactpro.th2.check1.rule import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.configuration.Check1Configuration +import com.exactpro.th2.check1.entities.RequestAdaptor import com.exactpro.th2.check1.entities.RuleConfiguration import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.exception.RuleCreationException @@ -28,7 +29,6 @@ import com.exactpro.th2.check1.rule.nomessage.NoMessageCheckTask import com.exactpro.th2.check1.rule.sequence.SequenceCheckRuleTask import com.exactpro.th2.check1.rule.sequence.SilenceCheckTask import com.exactpro.th2.common.event.Event -import com.exactpro.th2.common.grpc.Checkpoint import com.exactpro.th2.common.grpc.ComparisonSettings import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.EventBatch @@ -54,14 +54,14 @@ class RuleFactory( private val timePrecision = configuration.timePrecision private val decimalPrecision = configuration.decimalPrecision - fun createCheckRule(request: CheckRuleRequest): CheckRuleTask = + fun createCheckRule(request: CheckRuleRequest, isChainIdExist: Boolean): CheckRuleTask = ruleCreation(request.parentEventId) { checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } - check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias + check(sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(RequestAdaptor.from(request), sessionKey, isChainIdExist) } check(request.kindCase != CheckRuleRequest.KindCase.KIND_NOT_SET) { "Either old filter or root filter must be set" @@ -97,14 +97,14 @@ class RuleFactory( } } - fun createSequenceCheckRule(request: CheckSequenceRuleRequest): SequenceCheckRuleTask = + fun createSequenceCheckRule(request: CheckSequenceRuleRequest, isChainIdExist: Boolean): SequenceCheckRuleTask = ruleCreation(request.parentEventId) { checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } - check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias + check(sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(RequestAdaptor.from(request), sessionKey, isChainIdExist) } check((request.messageFiltersList.isEmpty() && request.rootMessageFiltersList.isNotEmpty()) || (request.messageFiltersList.isNotEmpty() && request.rootMessageFiltersList.isEmpty())) { @@ -142,15 +142,15 @@ class RuleFactory( } } - fun createNoMessageCheckRule(request: NoMessageCheckRequest): NoMessageCheckTask = + fun createNoMessageCheckRule(request: NoMessageCheckRequest, isChainIdExist: Boolean): NoMessageCheckTask = ruleCreation(request.parentEventId) { checkAndCreateRule { check(request.hasParentEventId()) { "Parent event id can't be null" } val parentEventID: EventID = request.parentEventId - check(request.connectivityId.sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionAlias: String = request.connectivityId.sessionAlias + check(sessionAlias.isNotEmpty()) { "Session alias cannot be empty" } val sessionKey = SessionKey(sessionAlias, directionOrDefault(request.direction)) - checkMessageTimeout(request.messageTimeout) { checkCheckpoint(request.checkpoint, sessionKey) } + checkMessageTimeout(request.messageTimeout) { checkCheckpoint(RequestAdaptor.from(request), sessionKey, isChainIdExist) } val ruleConfiguration = RuleConfiguration( createTaskTimeout(request.timeout, request.messageTimeout), @@ -272,10 +272,18 @@ class RuleFactory( } } - private fun checkCheckpoint(checkpoint: Checkpoint, sessionKey: SessionKey) { - check(checkpoint !== Checkpoint.getDefaultInstance()) { "Request doesn't contain a checkpoint" } + private fun checkCheckpoint(requestAdaptor: RequestAdaptor, sessionKey: SessionKey, isChainIdExist: Boolean) { + if (requestAdaptor.chainId != null) { + check(isChainIdExist) { + "The request has an invalid chain ID or connectivity ID. Please use checkpoint instead of chain ID" + } + return // We should validate checkpoint only if the request doesn't contain a chain id + } + checkNotNull(requestAdaptor.checkpoint) { + "Request must contain a checkpoint, because the 'messageTimeout' is used and no chain ID is specified" + } with(sessionKey) { - val directionCheckpoint = checkpoint.sessionAliasToDirectionCheckpointMap[sessionAlias] + val directionCheckpoint = requestAdaptor.checkpoint.sessionAliasToDirectionCheckpointMap[sessionAlias] checkNotNull(directionCheckpoint) { "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" } val checkpointData = directionCheckpoint.directionToCheckpointDataMap[direction.number] checkNotNull(checkpointData) { "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/RuleFactoryTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/RuleFactoryTest.kt index 96aa613b..fc29377d 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/RuleFactoryTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/RuleFactoryTest.kt @@ -17,15 +17,20 @@ import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.configuration.Check1Configuration import com.exactpro.th2.check1.exception.RuleCreationException +import com.exactpro.th2.check1.grpc.ChainID import com.exactpro.th2.check1.grpc.CheckRuleRequest +import com.exactpro.th2.check1.util.assertThrowsWithMessages import com.exactpro.th2.common.event.EventUtils import com.exactpro.th2.common.grpc.Checkpoint +import com.exactpro.th2.common.grpc.ConnectionID import com.exactpro.th2.common.grpc.Direction import com.exactpro.th2.common.grpc.EventBatch import com.exactpro.th2.common.grpc.EventID import com.exactpro.th2.common.grpc.Message import com.exactpro.th2.common.grpc.MessageMetadata +import com.exactpro.th2.common.grpc.RootMessageFilter import com.exactpro.th2.common.message.message +import com.exactpro.th2.common.message.toTimestamp import com.exactpro.th2.common.schema.message.MessageRouter import com.nhaarman.mockitokotlin2.argumentCaptor import com.nhaarman.mockitokotlin2.spy @@ -34,14 +39,16 @@ import com.nhaarman.mockitokotlin2.verify import io.reactivex.Observable import org.junit.jupiter.api.Test import org.junit.jupiter.api.assertAll -import org.junit.jupiter.api.assertThrows +import org.junit.jupiter.api.assertDoesNotThrow +import java.time.Instant import kotlin.test.assertEquals +import kotlin.test.assertNotNull class RuleFactoryTest { private val clientStub: MessageRouter = spy { } @Test - fun `failed rule creation because one of required fields is empty`() { + fun `failed rule creation because session alias is empty`() { val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) .mergeMetadata(MessageMetadata.newBuilder() @@ -57,10 +64,356 @@ class RuleFactoryTest { .setParentEventId(EventID.newBuilder().setId("root").build()) .setCheckpoint(Checkpoint.newBuilder().setId(EventUtils.generateUUID()).build()).build() - assertThrows { - ruleFactory.createCheckRule(request) + assertThrowsWithMessages( + "An error occurred while creating rule", + "Session alias cannot be empty" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + + @Test + fun `success rule creation with missed checkpoint`() { + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias("test_alias") + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setChainId(ChainID.newBuilder().setId("test_chain_id")) + .build() + + val createCheckRule = assertDoesNotThrow { + ruleFactory.createCheckRule(request, true) } + assertNotNull(createCheckRule) { "Rule cannot be null" } + } + + @Test + fun `failed rule creation with missed checkpoint and invalid chain id`() { + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias("test_alias") + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setChainId(ChainID.newBuilder().setId("test_chain_id")) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "The request has an invalid chain ID or connectivity ID. Please use checkpoint instead of chain ID" + ) { ruleFactory.createCheckRule(request, false) } + + assertEvents() + } + + @Test + fun `success rule creation with missed chain id`() { + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias("test_alias") + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setCheckpoint( + Checkpoint.newBuilder() + .setId(EventUtils.generateUUID()) + .putSessionAliasToDirectionCheckpoint( + "test_alias", + Checkpoint.DirectionCheckpoint.newBuilder() + .putDirectionToCheckpointData( + Direction.FIRST.number, + Checkpoint.CheckpointData.newBuilder() + .setSequence(1) + .setTimestamp(Instant.now().toTimestamp()) + .build()) + .build()) + .build() + ) + .setDirection(Direction.FIRST) + .build() + + + val createCheckRule = assertDoesNotThrow { + ruleFactory.createCheckRule(request, false) + } + assertNotNull(createCheckRule) { "Rule cannot be null" } + } + + @Test + fun `failed rule creation because direction checkpoint is missed`() { + val sessionAlias = "diff_test_alias" + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias(sessionAlias) + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setCheckpoint( + Checkpoint.newBuilder() + .setId(EventUtils.generateUUID()) + .putSessionAliasToDirectionCheckpoint( + "test_alias", + Checkpoint.DirectionCheckpoint.newBuilder() + .putDirectionToCheckpointData( + Direction.FIRST.number, + Checkpoint.CheckpointData.newBuilder() + .setSequence(1) + .setTimestamp(Instant.now().toTimestamp()) + .build()) + .build()) + .build() + ) + .setDirection(Direction.FIRST) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "The checkpoint doesn't contain a direction checkpoint with session alias '$sessionAlias'" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + + @Test + fun `failed rule creation because checkpoint is missed`() { + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias("test_alias") + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setDirection(Direction.FIRST) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "Request must contain a checkpoint, because the 'messageTimeout' is used and no chain ID is specified" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + + @Test + fun `failed rule creation because checkpoint data is missed`() { + val sessionAlias = "test_alias" + val direction = Direction.SECOND + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias(sessionAlias) + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setCheckpoint( + Checkpoint.newBuilder() + .setId(EventUtils.generateUUID()) + .putSessionAliasToDirectionCheckpoint( + sessionAlias, + Checkpoint.DirectionCheckpoint.newBuilder() + .putDirectionToCheckpointData( + Direction.FIRST.number, + Checkpoint.CheckpointData.newBuilder() + .setSequence(1) + .setTimestamp(Instant.now().toTimestamp()) + .build()) + .build()) + .build() + ) + .setDirection(direction) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "The direction checkpoint doesn't contain a checkpoint data with direction '$direction'" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + + @Test + fun `failed rule creation because checkpoint data has incorrect sequence number`() { + val sessionAlias = "test_alias" + val sequence: Long = -1 + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias(sessionAlias) + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setCheckpoint( + Checkpoint.newBuilder() + .setId(EventUtils.generateUUID()) + .putSessionAliasToDirectionCheckpoint( + sessionAlias, + Checkpoint.DirectionCheckpoint.newBuilder() + .putDirectionToCheckpointData( + Direction.FIRST.number, + Checkpoint.CheckpointData.newBuilder() + .setSequence(sequence) + .setTimestamp(Instant.now().toTimestamp()) + .build()) + .build()) + .build() + ) + .setDirection(Direction.FIRST) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "The checkpoint data has incorrect sequence number '$sequence'" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + + @Test + fun `failed rule creation because checkpoint data missed timestamp`() { + val sessionAlias = "test_alias" + val streams = createStreams(AbstractCheckTaskTest.SESSION_ALIAS, Direction.FIRST, listOf( + message(AbstractCheckTaskTest.MESSAGE_TYPE, Direction.FIRST, AbstractCheckTaskTest.SESSION_ALIAS) + .mergeMetadata(MessageMetadata.newBuilder() + .putProperties("keyProp", "42") + .putProperties("notKeyProp", "2") + .build()) + .build() + )) + + val ruleFactory = RuleFactory(Check1Configuration(), streams, clientStub) + + val request = CheckRuleRequest.newBuilder() + .setParentEventId(EventID.newBuilder().setId("root").build()) + .setConnectivityId(ConnectionID.newBuilder() + .setSessionAlias(sessionAlias) + ) + .setRootFilter(RootMessageFilter.newBuilder() + .setMessageType("TestMsgType") + ) + .setMessageTimeout(5) + .setCheckpoint( + Checkpoint.newBuilder() + .setId(EventUtils.generateUUID()) + .putSessionAliasToDirectionCheckpoint( + sessionAlias, + Checkpoint.DirectionCheckpoint.newBuilder() + .putDirectionToCheckpointData( + Direction.FIRST.number, + Checkpoint.CheckpointData.newBuilder() + .setSequence(1) + .build()) + .build()) + .build() + ) + .setDirection(Direction.FIRST) + .build() + + assertThrowsWithMessages( + "An error occurred while creating rule", + "The checkpoint data doesn't contain timestamp" + ) { ruleFactory.createCheckRule(request, true) } + + assertEvents() + } + private fun assertEvents() { val eventBatches = awaitEventBatchRequest(1000L, 1) val eventList = eventBatches.flatMap(EventBatch::getEventsList) assertAll({ diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt index d186adc9..dfe1efa0 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/sequence/TestSequenceCheckTaskWithSilenceCheck.kt @@ -62,7 +62,7 @@ class TestSequenceCheckTaskWithSilenceCheck : AbstractCheckTaskTest() { .setPreFilter(preFilter) .setParentEventId(parentId) .build() - val sequenceRule = factory.createSequenceCheckRule(request) + val sequenceRule = factory.createSequenceCheckRule(request, true) val silenceCheck = factory.createSilenceCheck(request, 1000) sequenceRule.subscribeNextTask(silenceCheck) sequenceRule.begin() @@ -109,7 +109,7 @@ class TestSequenceCheckTaskWithSilenceCheck : AbstractCheckTaskTest() { .setPreFilter(preFilter) .setParentEventId(parentId) .build() - val sequenceRule = factory.createSequenceCheckRule(request) + val sequenceRule = factory.createSequenceCheckRule(request, true) val silenceCheck = factory.createSilenceCheck(request, 1000) sequenceRule.subscribeNextTask(silenceCheck) sequenceRule.begin() @@ -161,9 +161,9 @@ class TestSequenceCheckTaskWithSilenceCheck : AbstractCheckTaskTest() { .putFields("B", ValueFilter.newBuilder().setOperation(FilterOperation.EQUAL).setSimpleFilter("2").build()) ).build()) .build() - val sequenceRule = factory.createSequenceCheckRule(request) + val sequenceRule = factory.createSequenceCheckRule(request, true) val silenceCheck = factory.createSilenceCheck(request, 1000) - val anotherRule = factory.createSequenceCheckRule(anotherRequest) + val anotherRule = factory.createSequenceCheckRule(anotherRequest, true) sequenceRule.subscribeNextTask(silenceCheck) sequenceRule.begin() silenceCheck.subscribeNextTask(anotherRule) @@ -216,7 +216,7 @@ class TestSequenceCheckTaskWithSilenceCheck : AbstractCheckTaskTest() { .setParentEventId(parentId) .build() val silenceCheck = factory.createSilenceCheck(request, 1000) - val sequenceRule = factory.createSequenceCheckRule(request.toBuilder().setDescription("2").build()) + val sequenceRule = factory.createSequenceCheckRule(request.toBuilder().setDescription("2").build(), true) silenceCheck.begin() silenceCheck.subscribeNextTask(sequenceRule) diff --git a/src/test/kotlin/com/exactpro/th2/check1/util/Utils.kt b/src/test/kotlin/com/exactpro/th2/check1/util/Utils.kt index 02bec5cc..6c6082db 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/util/Utils.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/util/Utils.kt @@ -17,6 +17,8 @@ import com.exactpro.th2.common.event.bean.VerificationEntry import com.exactpro.th2.common.event.bean.VerificationStatus import com.exactpro.th2.common.grpc.FilterOperation import com.exactpro.th2.common.grpc.MetadataFilter +import org.junit.jupiter.api.assertThrows +import kotlin.test.assertEquals fun String.toSimpleFilter(op: FilterOperation, key: Boolean = false): MetadataFilter.SimpleFilter = MetadataFilter.SimpleFilter.newBuilder() .setOperation(op) @@ -31,4 +33,15 @@ fun createVerificationEntry(status: VerificationStatus): VerificationEntry = Ver fun createVerificationEntry(vararg verificationEntries: Pair): VerificationEntry = VerificationEntry().apply { fields = linkedMapOf(*verificationEntries) +} + +inline fun assertThrowsWithMessages(vararg exceptionMessages: String?, crossinline action: () -> Unit) { + val exception = assertThrows { + action() + } + var currentException: Throwable? = exception + for (exceptionMessage in exceptionMessages) { + assertEquals(exceptionMessage, currentException?.message) + currentException = currentException?.cause + } } \ No newline at end of file From 340dffc0210f7d64bdf717151092e44cf1bc0d64 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Mon, 29 Nov 2021 18:55:58 +0300 Subject: [PATCH 33/41] [TH2-2764] Add support for null markers in message --- README.md | 1 + build.gradle | 2 +- .../check1/event/VerificationEntryUtils.java | 14 +- .../th2/check1/rule/AbstractCheckTask.kt | 3 +- .../event/TestVerificationEntryUtils.kt | 40 +++++- .../th2/check1/util/TestVerificationUtil.kt | 120 ++++++++++-------- 6 files changed, 120 insertions(+), 60 deletions(-) diff --git a/README.md b/README.md index d36409cb..d4dae562 100644 --- a/README.md +++ b/README.md @@ -162,6 +162,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` + Migrated `sailfish-utils` version from `3.9.1` to `3.10.2` + Fixed conversion of `null` values + + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value + Fixed setting of the `failUnexpected` parameter while converting a message filter + Migrated `sailfish-core` version to `3.2.1752` diff --git a/build.gradle b/build.gradle index e982a123..136e0abb 100644 --- a/build.gradle +++ b/build.gradle @@ -168,7 +168,7 @@ dependencies { api platform('com.exactpro.th2:bom:3.0.0') implementation 'com.exactpro.th2:grpc-check1:3.5.0' implementation 'com.exactpro.th2:common:3.28.0' - implementation 'com.exactpro.th2:sailfish-utils:3.10.2' + implementation 'com.exactpro.th2:sailfish-utils:3.11.0' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" diff --git a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java index 756914df..b7b41c92 100644 --- a/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java +++ b/src/main/java/com/exactpro/th2/check1/event/VerificationEntryUtils.java @@ -20,6 +20,8 @@ import java.util.Objects; import java.util.stream.Collectors; +import org.jetbrains.annotations.Nullable; + import com.exactpro.sf.aml.scriptutil.StaticUtil.IFilter; import com.exactpro.sf.comparison.ComparisonResult; import com.exactpro.sf.comparison.Formatter; @@ -28,12 +30,13 @@ import com.exactpro.th2.common.event.bean.VerificationStatus; import com.exactpro.th2.common.grpc.FilterOperation; import com.exactpro.th2.sailfish.utils.filter.IOperationFilter; +import com.exactpro.th2.sailfish.utils.filter.util.FilterUtils; public class VerificationEntryUtils { public static VerificationEntry createVerificationEntry(ComparisonResult result) { VerificationEntry verificationEntry = new VerificationEntry(); - verificationEntry.setActual(Objects.toString(result.getActual(), null)); + verificationEntry.setActual(convertActual(result)); verificationEntry.setExpected(convertExpectedResult(result)); verificationEntry.setStatus(toVerificationStatus(result.getStatus())); verificationEntry.setKey(result.isKey()); @@ -54,6 +57,15 @@ public static VerificationEntry createVerificationEntry(ComparisonResult result) return verificationEntry; } + @Nullable + private static String convertActual(ComparisonResult result) { + Object actual = result.getActual(); + if (actual == FilterUtils.NULL_VALUE) { + return null; + } + return Objects.toString(actual, null); + } + private static String resolveOperation(ComparisonResult result) { Object expected = result.getExpected(); if (expected instanceof IFilter) { diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 18113c6d..4ad2ca33 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -49,6 +49,7 @@ import com.exactpro.th2.common.message.toReadableBodyCollection import com.exactpro.th2.common.schema.message.MessageRouter import com.exactpro.th2.sailfish.utils.FilterSettings import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter +import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter.createParameters import com.google.protobuf.TextFormat.shortDebugString import com.google.protobuf.Timestamp import com.google.protobuf.util.Durations @@ -85,7 +86,7 @@ abstract class AbstractCheckTask( protected var handledMessageCounter: Long = 0 - protected val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null) + protected val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, createParameters().setUseMarkerForNullsInMessage(true)) protected val rootEvent: Event = Event.from(submitTime).description(description) private val sequenceSubject = SingleSubject.create() diff --git a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt index 88ecef6d..88d9d73f 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt @@ -17,6 +17,7 @@ import com.exactpro.sf.comparison.ComparatorSettings import com.exactpro.sf.comparison.MessageComparator import com.exactpro.th2.check1.util.VerificationUtil import com.exactpro.th2.common.event.bean.VerificationEntry +import com.exactpro.th2.common.grpc.FilterOperation import com.exactpro.th2.common.grpc.ListValueFilter import com.exactpro.th2.common.grpc.MessageFilter import com.exactpro.th2.common.grpc.RootMessageFilter @@ -24,9 +25,11 @@ import com.exactpro.th2.common.grpc.Value import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.message.message import com.exactpro.th2.common.message.messageFilter +import com.exactpro.th2.common.value.nullValue import com.exactpro.th2.common.value.toValue import com.exactpro.th2.common.value.toValueFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter +import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter.createParameters import com.fasterxml.jackson.databind.ObjectMapper import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test @@ -38,7 +41,42 @@ import org.junit.jupiter.params.provider.MethodSource import java.util.stream.Stream class TestVerificationEntryUtils { - private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null) + private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, + createParameters().setUseMarkerForNullsInMessage(true) + ) + + @Test + fun `null value in message`() { + val filter = RootMessageFilter.newBuilder() + .setMessageType("test") + .setMessageFilter( + MessageFilter.newBuilder() + .putFields("A", ValueFilter.newBuilder().setOperation(FilterOperation.EMPTY).build()) + ).build() + + val message = message("test").putFields("A", nullValue()).putFields("B", nullValue()).build() + + val expected = converter.fromProtoFilter(filter.messageFilter, "test") + val actual = converter.fromProtoMessage(message, false) + val settings = ComparatorSettings().apply { + isKeepResultGroupOrder = true + } + + val result = MessageComparator.compare(actual, expected, settings).assertNotNull { + "Result must not be null" + } + val entry = VerificationEntryUtils.createVerificationEntry(result) + entry.fields["A"].assertNotNull { "Field A must be set in entry: ${entry.toDebugString()}" } + .also { + Assertions.assertEquals("#", it.expected) { "Expected value is different in entry: ${it.toDebugString()}" } + Assertions.assertNull(it.actual) { "Actual value must be null in entry: ${it.toDebugString()}" } + } + entry.fields["B"].assertNotNull { "Field B must be set in entry: ${entry.toDebugString()}" } + .also { + Assertions.assertNull(it.expected) { "Expected value must be null in entry: ${it.toDebugString()}" } + Assertions.assertNull(it.actual) { "Actual value must be null in entry: ${it.toDebugString()}" } + } + } @Test fun `key field in reordered collection`() { diff --git a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt index 90db7b1f..d8acb2d6 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt @@ -27,6 +27,7 @@ import com.exactpro.th2.common.grpc.Value import com.exactpro.th2.common.grpc.ValueFilter import com.exactpro.th2.common.message.message import com.exactpro.th2.common.message.messageFilter +import com.exactpro.th2.common.value.nullValue import com.exactpro.th2.common.value.toValue import com.exactpro.th2.common.value.toValueFilter import com.exactpro.th2.sailfish.utils.ProtoToIMessageConverter @@ -34,6 +35,7 @@ import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.Arguments +import org.junit.jupiter.params.provider.Arguments.arguments import org.junit.jupiter.params.provider.MethodSource import java.util.stream.Stream @@ -57,15 +59,15 @@ internal class TestVerificationUtil { @MethodSource("failUnexpectedByFieldsAndMessages") fun `fail unexpected test`(valueFilter: ValueFilter, value: Value) { val filter: RootMessageFilter = RootMessageFilter.newBuilder() - .setMessageType("Test") - .setMessageFilter(MessageFilter.newBuilder() - .putFields("A", valueFilter) - .setComparisonSettings(ComparisonSettings.newBuilder() - .setFailUnexpected(FailUnexpected.FIELDS_AND_MESSAGES) - .build() - ) - .build()) - .build() + .setMessageType("Test") + .setMessageFilter(MessageFilter.newBuilder() + .putFields("A", valueFilter) + .setComparisonSettings(ComparisonSettings.newBuilder() + .setFailUnexpected(FailUnexpected.FIELDS_AND_MESSAGES) + .build() + ) + .build()) + .build() val actual = message("Test").apply { putFields("A", value) @@ -78,9 +80,9 @@ internal class TestVerificationUtil { val actualIMessage = converter.fromProtoMessage(actual, false) val filterIMessage = converter.fromProtoFilter(filter.messageFilter, filter.messageType) val result = MessageComparator.compare( - actualIMessage, - filterIMessage, - settings + actualIMessage, + filterIMessage, + settings ) Assertions.assertNotNull(result) { "Result cannot be null" } @@ -88,53 +90,59 @@ internal class TestVerificationUtil { } companion object { - private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null) + private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, + ProtoToIMessageConverter.createParameters().setUseMarkerForNullsInMessage(true) + ) @JvmStatic fun failUnexpectedByFieldsAndMessages(): Stream = Stream.of( - Arguments.arguments( - listOf( - messageFilter().putFields("A1", "1".toValueFilter()) - ).toValueFilter(), - listOf( - message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build() - ).toValue() - ), - Arguments.arguments( - messageFilter().putFields("A1", "1".toValueFilter()).toValueFilter(), - message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build().toValue() - ), - Arguments.arguments( - messageFilter().putFields( - "A1", listOf("1", "2").toValueFilter() - ).toValueFilter(), - message().putFields( - "A1", - listOf("1", "2", "3").toValue() - ).build().toValue() - ), - Arguments.arguments( - messageFilter().putFields( - "A1", - messageFilter().putFields("A2", "1".toValueFilter()).toValueFilter() - ).toValueFilter(), - message().putFields( - "A1", - message() - .putFields("A2", "1".toValue()) - .putFields("B2", "2".toValue()) - .build().toValue() - ).build().toValue() - ), - Arguments.arguments( - listOf( - "1".toValueFilter() - ).toValueFilter(), - listOf( - "1".toValue(), - "2".toValue() - ).toValue() - ) + arguments( + listOf( + messageFilter().putFields("A1", "1".toValueFilter()) + ).toValueFilter(), + listOf( + message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build() + ).toValue() + ), + arguments( + messageFilter().putFields("A1", "1".toValueFilter()).toValueFilter(), + message().putFields("A1", "1".toValue()).putFields("B1", "2".toValue()).build().toValue() + ), + arguments( + messageFilter().putFields( + "A1", listOf("1", "2").toValueFilter() + ).toValueFilter(), + message().putFields( + "A1", + listOf("1", "2", "3").toValue() + ).build().toValue() + ), + arguments( + messageFilter().putFields( + "A1", + messageFilter().putFields("A2", "1".toValueFilter()).toValueFilter() + ).toValueFilter(), + message().putFields( + "A1", + message() + .putFields("A2", "1".toValue()) + .putFields("B2", "2".toValue()) + .build().toValue() + ).build().toValue() + ), + arguments( + listOf( + "1".toValueFilter() + ).toValueFilter(), + listOf( + "1".toValue(), + "2".toValue() + ).toValue() + ), + arguments( + messageFilter().putFields("A", 42.toValueFilter()).toValueFilter(), + message().putFields("A", 42.toValue()).putFields("B", nullValue()).toValue() + ) ) } } \ No newline at end of file From ce69c8a0d68571a4fb99b23944584f7d7598341b Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Tue, 30 Nov 2021 14:05:29 +0300 Subject: [PATCH 34/41] [TH2-2764] Make converter static. Correct readme --- README.md | 2 +- .../com/exactpro/th2/check1/rule/AbstractCheckTask.kt | 6 +++--- .../com/exactpro/th2/check1/rule/check/CheckRuleTask.kt | 4 ++-- .../th2/check1/rule/nomessage/NoMessageCheckTask.kt | 4 ++-- .../th2/check1/rule/sequence/SequenceCheckRuleTask.kt | 8 ++++---- .../exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt | 5 ++--- .../th2/check1/event/TestVerificationEntryUtils.kt | 5 ++--- .../com/exactpro/th2/check1/util/TestVerificationUtil.kt | 5 ++--- 8 files changed, 18 insertions(+), 21 deletions(-) diff --git a/README.md b/README.md index d4dae562..2748d148 100644 --- a/README.md +++ b/README.md @@ -160,7 +160,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type #### Changed: + Migrated `common` version from `3.26.4` to `3.28.0` + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.10.2` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.11.0` + Fixed conversion of `null` values + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 4ad2ca33..533699e2 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -85,8 +85,6 @@ abstract class AbstractCheckTask( private val taskTimeout: TaskTimeout = ruleConfiguration.taskTimeout protected var handledMessageCounter: Long = 0 - - protected val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, createParameters().setUseMarkerForNullsInMessage(true)) protected val rootEvent: Event = Event.from(submitTime).description(description) private val sequenceSubject = SingleSubject.create() @@ -539,6 +537,8 @@ abstract class AbstractCheckTask( companion object { const val DEFAULT_SEQUENCE = Long.MIN_VALUE private val RESPONSE_EXECUTOR = ForkJoinPool.commonPool() + @JvmField + val CONVERTER = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, createParameters().setUseMarkerForNullsInMessage(true)) } protected fun RootMessageFilter.metadataFilterOrNull(): MetadataFilter? = @@ -649,7 +649,7 @@ abstract class AbstractCheckTask( } private fun Observable.mapToMessageContainer(): Observable = - map { message -> MessageContainer(message, converter.fromProtoMessage(message, false)) } + map { message -> MessageContainer(message, CONVERTER.fromProtoMessage(message, false)) } /** * Filters incoming {@link StreamContainer} via session alias and then diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt index 1ae52aca..b94dd543 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/check/CheckRuleTask.kt @@ -46,12 +46,12 @@ class CheckRuleTask( ) : AbstractCheckTask(ruleConfiguration, startTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val messageFilter: SailfishFilter = SailfishFilter( - converter.fromProtoPreFilter(protoMessageFilter), + CONVERTER.fromProtoPreFilter(protoMessageFilter), protoMessageFilter.toCompareSettings() ) private val metadataFilter: SailfishFilter? = protoMessageFilter.metadataFilterOrNull()?.let { SailfishFilter( - converter.fromMetadataFilter(it, METADATA_MESSAGE_NAME), + CONVERTER.fromMetadataFilter(it, METADATA_MESSAGE_NAME), it.toComparisonSettings() ) } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt index 3dacd01e..c580df8a 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/nomessage/NoMessageCheckTask.kt @@ -45,13 +45,13 @@ class NoMessageCheckTask( private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( - converter.fromProtoPreFilter(protoPreMessageFilter), + CONVERTER.fromProtoPreFilter(protoPreMessageFilter), protoPreMessageFilter.toCompareSettings() ) private val metadataPreFilter: SailfishFilter? = protoPreMessageFilter.metadataFilterOrNull()?.let { SailfishFilter( - converter.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), + CONVERTER.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), it.toComparisonSettings() ) } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt index af10c5fa..5b1810fc 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SequenceCheckRuleTask.kt @@ -71,12 +71,12 @@ class SequenceCheckRuleTask( private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( - converter.fromProtoPreFilter(protoPreMessageFilter), + CONVERTER.fromProtoPreFilter(protoPreMessageFilter), protoPreMessageFilter.toCompareSettings() ) private val metadataPreFilter: SailfishFilter? = protoPreMessageFilter.metadataFilterOrNull()?.let { SailfishFilter( - converter.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), + CONVERTER.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), it.toComparisonSettings() ) } @@ -104,9 +104,9 @@ class SequenceCheckRuleTask( messageFilters = protoMessageFilters.map { MessageFilterContainer( it, - SailfishFilter(converter.fromProtoPreFilter(it), it.toCompareSettings()), + SailfishFilter(CONVERTER.fromProtoPreFilter(it), it.toCompareSettings()), it.metadataFilterOrNull()?.let { metadataFilter -> - SailfishFilter(converter.fromMetadataFilter(metadataFilter, VerificationUtil.METADATA_MESSAGE_NAME), + SailfishFilter(CONVERTER.fromMetadataFilter(metadataFilter, VerificationUtil.METADATA_MESSAGE_NAME), metadataFilter.toComparisonSettings()) } ) diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt index 7fc3b478..0fdfb386 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/sequence/SilenceCheckTask.kt @@ -16,7 +16,6 @@ package com.exactpro.th2.check1.rule.sequence import com.exactpro.th2.check1.SessionKey import com.exactpro.th2.check1.StreamContainer import com.exactpro.th2.check1.entities.RuleConfiguration -import com.exactpro.th2.check1.entities.TaskTimeout import com.exactpro.th2.check1.grpc.PreFilter import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.rule.MessageContainer @@ -46,12 +45,12 @@ class SilenceCheckTask( ) : AbstractCheckTask(ruleConfiguration, submitTime, sessionKey, parentEventID, messageStream, eventBatchRouter) { private val protoPreMessageFilter: RootMessageFilter = protoPreFilter.toRootMessageFilter() private val messagePreFilter = SailfishFilter( - converter.fromProtoPreFilter(protoPreMessageFilter), + CONVERTER.fromProtoPreFilter(protoPreMessageFilter), protoPreMessageFilter.toCompareSettings() ) private val metadataPreFilter: SailfishFilter? = protoPreMessageFilter.metadataFilterOrNull()?.let { SailfishFilter( - converter.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), + CONVERTER.fromMetadataFilter(it, VerificationUtil.METADATA_MESSAGE_NAME), it.toComparisonSettings() ) } diff --git a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt index 88d9d73f..61d13183 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/event/TestVerificationEntryUtils.kt @@ -15,6 +15,7 @@ package com.exactpro.th2.check1.event import com.exactpro.sf.comparison.ComparatorSettings import com.exactpro.sf.comparison.MessageComparator +import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.check1.util.VerificationUtil import com.exactpro.th2.common.event.bean.VerificationEntry import com.exactpro.th2.common.grpc.FilterOperation @@ -41,9 +42,7 @@ import org.junit.jupiter.params.provider.MethodSource import java.util.stream.Stream class TestVerificationEntryUtils { - private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, - createParameters().setUseMarkerForNullsInMessage(true) - ) + private val converter = AbstractCheckTask.CONVERTER @Test fun `null value in message`() { diff --git a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt index d8acb2d6..9d71702a 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/util/TestVerificationUtil.kt @@ -17,6 +17,7 @@ import com.exactpro.sf.comparison.ComparatorSettings import com.exactpro.sf.comparison.ComparisonUtil import com.exactpro.sf.comparison.MessageComparator import com.exactpro.sf.scriptrunner.StatusType +import com.exactpro.th2.check1.rule.AbstractCheckTask import com.exactpro.th2.common.grpc.ComparisonSettings import com.exactpro.th2.common.grpc.FailUnexpected import com.exactpro.th2.common.grpc.FilterOperation @@ -90,9 +91,7 @@ internal class TestVerificationUtil { } companion object { - private val converter = ProtoToIMessageConverter(VerificationUtil.FACTORY_PROXY, null, null, - ProtoToIMessageConverter.createParameters().setUseMarkerForNullsInMessage(true) - ) + private val converter = AbstractCheckTask.CONVERTER @JvmStatic fun failUnexpectedByFieldsAndMessages(): Stream = Stream.of( From 18f5bce52c78619185bb31b9fb18c2502387f25e Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Thu, 2 Dec 2021 12:42:24 +0400 Subject: [PATCH 35/41] [TH2-2773] Updated sailfish-utils version (#109) --- README.md | 2 +- build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 2748d148..e0d9481e 100644 --- a/README.md +++ b/README.md @@ -160,7 +160,7 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type #### Changed: + Migrated `common` version from `3.26.4` to `3.28.0` + Migrated `grpc-check1` version from `3.4.2` to `3.5.0` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.11.0` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.11.1` + Fixed conversion of `null` values + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value diff --git a/build.gradle b/build.gradle index 136e0abb..36cd2146 100644 --- a/build.gradle +++ b/build.gradle @@ -168,7 +168,7 @@ dependencies { api platform('com.exactpro.th2:bom:3.0.0') implementation 'com.exactpro.th2:grpc-check1:3.5.0' implementation 'com.exactpro.th2:common:3.28.0' - implementation 'com.exactpro.th2:sailfish-utils:3.11.0' + implementation 'com.exactpro.th2:sailfish-utils:3.11.1' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" From 54ee4d11b0a075e9f6a56e21be52274d030c99f8 Mon Sep 17 00:00:00 2001 From: Oleg Smirnov Date: Fri, 3 Dec 2021 17:34:29 +0300 Subject: [PATCH 36/41] [TH2-2764] Allow checking for exact null value in message. Update kotlin version (#111) --- README.md | 7 ++++--- build.gradle | 9 +++++---- 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index e0d9481e..780cd724 100644 --- a/README.md +++ b/README.md @@ -158,11 +158,12 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + New parameter `hint` for verification event which is used to display the reason for the failed field comparison. For example the type mismatch of the compared values #### Changed: -+ Migrated `common` version from `3.26.4` to `3.28.0` -+ Migrated `grpc-check1` version from `3.4.2` to `3.5.0` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.11.1` ++ Migrated `common` version from `3.26.4` to `3.31.2` ++ Migrated `grpc-check1` version from `3.4.2` to `3.5.1` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.12.0` + Fixed conversion of `null` values + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all + + Allow checking for exact `null` value in message. + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value + Fixed setting of the `failUnexpected` parameter while converting a message filter + Migrated `sailfish-core` version to `3.2.1752` diff --git a/build.gradle b/build.gradle index 36cd2146..bee42f33 100644 --- a/build.gradle +++ b/build.gradle @@ -1,6 +1,6 @@ plugins { id 'com.palantir.docker' version '0.25.0' - id 'org.jetbrains.kotlin.jvm' version '1.3.72' + id 'org.jetbrains.kotlin.jvm' version '1.5.30' id "io.github.gradle-nexus.publish-plugin" version "1.0.0" id 'signing' id 'java-library' @@ -166,9 +166,9 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') - implementation 'com.exactpro.th2:grpc-check1:3.5.0' - implementation 'com.exactpro.th2:common:3.28.0' - implementation 'com.exactpro.th2:sailfish-utils:3.11.1' + implementation 'com.exactpro.th2:grpc-check1:3.5.1' + implementation 'com.exactpro.th2:common:3.31.2' + implementation 'com.exactpro.th2:sailfish-utils:3.12.0' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" @@ -179,6 +179,7 @@ dependencies { implementation('io.prometheus:simpleclient') { because('metrics from messages and rules') } + implementation 'org.jetbrains.kotlin:kotlin-reflect' testImplementation 'org.junit.jupiter:junit-jupiter:5.6.2' testImplementation 'org.jetbrains.kotlin:kotlin-test-junit' From fb417960379b1bf33580e096ffaf4e7c70d94812 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Fri, 3 Dec 2021 20:10:30 +0300 Subject: [PATCH 37/41] [TH2-2457] Add description for required and optional parameters in check requests --- README.md | 41 +++++++++++++++++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) diff --git a/README.md b/README.md index 780cd724..7bd80e6b 100644 --- a/README.md +++ b/README.md @@ -24,6 +24,47 @@ Available requests are described in [this repository](https://gitlab.exactpro.co - CheckRuleRequest - get message filter from request and check it with messages in the cache or await specified time in case of empty cache or message absence. - NoMessageCheckRequest - prefilters messages and verifies that no other messages have been received. +## Request parameters + +### Common + +#### Required + +* **parent_event_id** - all events generated by the rule will be attached to that event +* **connectivity_id** (the `session_alias` inside `connectivity_id` must not be empty) + +#### Optional + +* **direction** - the direction of messages to be checked by rule. By default, it is _FIRST_ +* **chain_id** - the id to connect rules (rule starts checking after the previous one in the chain). Considers **connectivity_id** +* **description** - the description that will be added to the root event produced by the rule +* **timeout** - defines the allowed timeout for messages matching by real time. If not set the default value from check1 settings will be taken +* **message_timeout** - defines the allowed timeout for messages matching by the time they were received. +* **checkpoint** (must be set if `message_timeout` is used and no valid `chain_id` has been provided) + +### CheckRuleRequest + +#### Required + +* **root_filter** or **filter** (please note, that the `filter` parameter is deprecated and will be removed in the future releases) + +### CheckSequenceRuleRequest + +#### Required + +* **root_message_filters** or **message_filters** with at least one filter + (please note, that the `message_filters` parameter is deprecated and will be removed in the future releases) + +#### Optional +* **pre_filter** - pre-filtering for messages. Only messages passed the filter will be checked by main filters. +* **check_order** - enables order validation in message's collections +* **silence_check** - enables auto-check for messages that match the `pre_filter` after the rule has finished + +### NoMessageCheckRequest + +#### Optional +* **pre_filter** pre-filtering for messages that should not be received. + ## Quick start General view of the component will look like this: From 92e57f758fa0678740c6026413429e9eda51f66f Mon Sep 17 00:00:00 2001 From: Oleg Smirnov Date: Sat, 4 Dec 2021 14:38:53 +0300 Subject: [PATCH 38/41] [TH2-2758] Update common version (#113) Add support for null value filter when converting filter to table Co-authored-by: Yaroslav Kuznetsov --- README.md | 4 ++-- build.gradle | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/README.md b/README.md index 780cd724..8fac66a0 100644 --- a/README.md +++ b/README.md @@ -158,9 +158,9 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + New parameter `hint` for verification event which is used to display the reason for the failed field comparison. For example the type mismatch of the compared values #### Changed: -+ Migrated `common` version from `3.26.4` to `3.31.2` ++ Migrated `common` version from `3.26.4` to `3.31.3` + Migrated `grpc-check1` version from `3.4.2` to `3.5.1` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.12.0` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.12.1` + Fixed conversion of `null` values + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all + Allow checking for exact `null` value in message. diff --git a/build.gradle b/build.gradle index bee42f33..8e6fb8db 100644 --- a/build.gradle +++ b/build.gradle @@ -167,8 +167,8 @@ signing { dependencies { api platform('com.exactpro.th2:bom:3.0.0') implementation 'com.exactpro.th2:grpc-check1:3.5.1' - implementation 'com.exactpro.th2:common:3.31.2' - implementation 'com.exactpro.th2:sailfish-utils:3.12.0' + implementation 'com.exactpro.th2:common:3.31.3' + implementation 'com.exactpro.th2:sailfish-utils:3.12.1' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" From 7c479eadffa8eb6323d150f23c4ba3f1bc8680d4 Mon Sep 17 00:00:00 2001 From: "oleg.smirnov" Date: Mon, 6 Dec 2021 10:50:52 +0300 Subject: [PATCH 39/41] [TH2-2457] Correct mistakes in readme file --- README.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index 7bd80e6b..1db66385 100644 --- a/README.md +++ b/README.md @@ -18,9 +18,9 @@ Available requests are described in [this repository](https://gitlab.exactpro.co - CheckSequenceRuleRequest - prefilters the messages and verify all of them by filter. Order checking configured from request. Depending on the request and check1 configuration **SilenceCheckRule** can be added after the CheckSequenceRule. - It verifies that there were not any messages matches the pre-filter in original request. + It verifies that there were not any messages matching the pre-filter in the original request. It awaits for realtime timeout that is equal to clean-up timeout. - Report about unexpected messages only after the timeout exceeds. Reports nothing if any task is added to the chain. + Reports about unexpected messages only after the timeout is exceeded. Reports nothing if any task is added to the chain. - CheckRuleRequest - get message filter from request and check it with messages in the cache or await specified time in case of empty cache or message absence. - NoMessageCheckRequest - prefilters messages and verifies that no other messages have been received. @@ -56,7 +56,7 @@ Available requests are described in [this repository](https://gitlab.exactpro.co (please note, that the `message_filters` parameter is deprecated and will be removed in the future releases) #### Optional -* **pre_filter** - pre-filtering for messages. Only messages passed the filter will be checked by main filters. +* **pre_filter** - pre-filtering for messages. Only messages that passed the filter will be checked by the main filters. * **check_order** - enables order validation in message's collections * **silence_check** - enables auto-check for messages that match the `pre_filter` after the rule has finished @@ -150,13 +150,13 @@ Defines a default behavior for creating CheckSequenceRule if `silence_check` par The time precision is used to compare two time values. It is based on the `ISO-8601` duration format `PnDTnHnMn.nS` with days considered to be exactly 24 hours. Additional information can be found [here](https://docs.oracle.com/en/java/javase/11/docs/api/java.base/java/time/Duration.html#parse(java.lang.CharSequence)) #### decimal-precision -The decimal precision is used to compare two number values. Can be specified in number or string format. For example `0.0001`, `0.125`, `125E-3` +The decimal precision is used to compare the value of two numbers. Can be specified in number or string format. For example `0.0001`, `0.125`, `125E-3` ## Required pins The Check1 component has two types of pin: * gRPC server pin: it allows other components to connect via `com.exactpro.th2.check1.grpc.Check1Service` class. -* MQ pin: it is used for listening to parsed messages. You can link several sources with different directions and session alises to it. +* MQ pin: it is used for listening to parsed messages. You can link several sources with different directions and session aliases to it. ```yaml apiVersion: th2.exactpro.com/v1 @@ -194,8 +194,8 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + New configuration parameter `rule-execution-timeout` which is used if the user has not specified a timeout for the rule execution + Auto silence check after the CheckSequenceRule. + `auto-silence-check-after-sequence-rule` to setup a default behavior for CheckSequenceRule -+ New configuration parameter `time-precision` which is uses if the user has not specified a time precision -+ New configuration parameter `decimal-precision` which is uses if the user has not specified a number precision ++ New configuration parameter `time-precision` which is used if the user has not specified a time precision ++ New configuration parameter `decimal-precision` which is used if the user has not specified a number precision + New parameter `hint` for verification event which is used to display the reason for the failed field comparison. For example the type mismatch of the compared values #### Changed: From 4996456d35eedd5ee7f8cb1cde2cfafd5efa96b1 Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Tue, 7 Dec 2021 16:57:07 +0400 Subject: [PATCH 40/41] [TH2-2773] Updated sailfish-utils version (#119) * [TH2-2773] Updated sailfish-utils version * Added net parameter checkNullValueAsEmpty to Check1Configuration * [TH2-2773] Corrected README.md --- README.md | 13 ++++++-- build.gradle | 2 +- .../configuration/Check1Configuration.java | 7 ++++ .../th2/check1/entities/RuleConfiguration.kt | 3 +- .../th2/check1/rule/AbstractCheckTask.kt | 1 + .../exactpro/th2/check1/rule/RuleFactory.kt | 13 +++++--- .../check1/entities/RuleConfigurationTest.kt | 32 ++++++++++++++++--- .../th2/check1/rule/AbstractCheckTaskTest.kt | 9 +++++- 8 files changed, 66 insertions(+), 14 deletions(-) diff --git a/README.md b/README.md index f173651c..971dae3e 100644 --- a/README.md +++ b/README.md @@ -121,7 +121,8 @@ This block describes the configuration for check1. "rule-execution-timeout": 5000, "auto-silence-check-after-sequence-rule": false, "time-precision": "PT0.000000001S", - "decimal-precision": 0.00001 + "decimal-precision": 0.00001, + "check-null-value-as-empty": true } ``` @@ -152,6 +153,10 @@ The time precision is used to compare two time values. It is based on the `ISO-8 #### decimal-precision The decimal precision is used to compare the value of two numbers. Can be specified in number or string format. For example `0.0001`, `0.125`, `125E-3` +#### check-null-value-as-empty +`check-null-value-as-empty` is used for `EMPTY` and `NOT_EMPTY` operations to check if `NULL_VALUE` value is empty. For example, if the `checkNullValueAsEmpty` parameter is: ++ `true`, then `NULL_VALUE` is equal to `EMPTY`, otherwise `NULL_VALUE` is equal to `NOT_EMPTY` + ## Required pins The Check1 component has two types of pin: @@ -197,14 +202,16 @@ The `th2_check1_active_tasks_number` metric separate rules with label `rule_type + New configuration parameter `time-precision` which is used if the user has not specified a time precision + New configuration parameter `decimal-precision` which is used if the user has not specified a number precision + New parameter `hint` for verification event which is used to display the reason for the failed field comparison. For example the type mismatch of the compared values ++ New configuration parameter `check-null-value-as-empty` witch us used to configure the `EMPTY` and `NOT_EMPTY` operations #### Changed: + Migrated `common` version from `3.26.4` to `3.31.3` + Migrated `grpc-check1` version from `3.4.2` to `3.5.1` -+ Migrated `sailfish-utils` version from `3.9.1` to `3.12.1` ++ Migrated `sailfish-utils` version from `3.9.1` to `3.12.2` + Fixed conversion of `null` values + Add marker for `null` values to determine whether the field was set with `null` value or was not set at all - + Allow checking for exact `null` value in message. + + Allow checking for exact `null` value in message + + Added new parameter `checkNullValueAsEmpty` in the `FilterSettings` + Corrected verification entry when the `null` value and string `"null"` looked the same for the expected value + Fixed setting of the `failUnexpected` parameter while converting a message filter + Migrated `sailfish-core` version to `3.2.1752` diff --git a/build.gradle b/build.gradle index 8e6fb8db..8549860f 100644 --- a/build.gradle +++ b/build.gradle @@ -168,7 +168,7 @@ dependencies { api platform('com.exactpro.th2:bom:3.0.0') implementation 'com.exactpro.th2:grpc-check1:3.5.1' implementation 'com.exactpro.th2:common:3.31.3' - implementation 'com.exactpro.th2:sailfish-utils:3.12.1' + implementation 'com.exactpro.th2:sailfish-utils:3.12.2' implementation "org.slf4j:slf4j-log4j12" implementation "org.slf4j:slf4j-api" diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index 3ff70b54..bf983bdc 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -49,6 +49,9 @@ public class Check1Configuration { @JsonDeserialize(using = DurationDeserializer.class) @JsonPropertyDescription("The default time precision value uses java duration format") private Duration timePrecision = Duration.parse("PT0.000000001S"); + + @JsonProperty(value = "check-null-value-as-empty") + private boolean checkNullValueAsEmpty = true; public int getMessageCacheSize() { return messageCacheSize; @@ -81,4 +84,8 @@ public double getDecimalPrecision() { public Duration getTimePrecision() { return timePrecision; } + + public boolean isCheckNullValueAsEmpty() { + return checkNullValueAsEmpty; + } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt b/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt index 451ff93b..26d7a125 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/entities/RuleConfiguration.kt @@ -20,7 +20,8 @@ data class RuleConfiguration( val description: String?, val timePrecision: Duration, val decimalPrecision: Double, - val maxEventBatchContentSize: Int + val maxEventBatchContentSize: Int, + val isCheckNullValueAsEmpty: Boolean ) { init { require(!timePrecision.isNegative) { "Time precision cannot be negative" } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt index 533699e2..ebdd121d 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTask.kt @@ -642,6 +642,7 @@ abstract class AbstractCheckTask( } else { ruleConfiguration.timePrecision } + isCheckNullValueAsEmpty = ruleConfiguration.isCheckNullValueAsEmpty } } diff --git a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt index 3e5037fb..30018ec0 100644 --- a/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt +++ b/src/main/kotlin/com/exactpro/th2/check1/rule/RuleFactory.kt @@ -53,6 +53,7 @@ class RuleFactory( private val defaultRuleExecutionTimeout = configuration.ruleExecutionTimeout private val timePrecision = configuration.timePrecision private val decimalPrecision = configuration.decimalPrecision + private val isCheckNullValueAsEmpty = configuration.isCheckNullValueAsEmpty fun createCheckRule(request: CheckRuleRequest, isChainIdExist: Boolean): CheckRuleTask = ruleCreation(request.parentEventId) { @@ -77,7 +78,8 @@ class RuleFactory( request.description, timePrecision, decimalPrecision, - maxEventBatchContentSize + maxEventBatchContentSize, + isCheckNullValueAsEmpty ) CheckRuleTask( @@ -120,7 +122,8 @@ class RuleFactory( request.description, timePrecision, decimalPrecision, - maxEventBatchContentSize + maxEventBatchContentSize, + isCheckNullValueAsEmpty ) SequenceCheckRuleTask( @@ -157,7 +160,8 @@ class RuleFactory( request.description, timePrecision, decimalPrecision, - maxEventBatchContentSize + maxEventBatchContentSize, + isCheckNullValueAsEmpty ) NoMessageCheckTask( @@ -192,7 +196,8 @@ class RuleFactory( request.description.takeIf(String::isNotEmpty), timePrecision, decimalPrecision, - maxEventBatchContentSize + maxEventBatchContentSize, + isCheckNullValueAsEmpty ) SilenceCheckTask( diff --git a/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt b/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt index 9502d4ed..355350e2 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/entities/RuleConfigurationTest.kt @@ -23,7 +23,12 @@ class RuleConfigurationTest { @Test fun `check that time precision is negative`() { val exception = assertThrows { - RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(-1), 0.005, 1) + RuleConfiguration( + TaskTimeout(0, 0), null, Duration.ofSeconds(-1), + 0.005, + 1, + true + ) } assertEquals(exception.message, "Time precision cannot be negative") } @@ -31,7 +36,12 @@ class RuleConfigurationTest { @Test fun `check that decimal precision is negative`() { val exception = assertThrows { - RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(1), -0.005, 1) + RuleConfiguration( + TaskTimeout(0, 0), null, Duration.ofSeconds(1), + -0.005, + 1, + true + ) } assertEquals(exception.message, "Decimal precision cannot be negative") } @@ -41,7 +51,14 @@ class RuleConfigurationTest { val maxEventBatchContentSize = -1 val exception = assertThrows { - RuleConfiguration(TaskTimeout(0, 0), null, Duration.ofSeconds(1), 0.005, maxEventBatchContentSize) + RuleConfiguration( + TaskTimeout(0, 0), + null, + Duration.ofSeconds(1), + 0.005, + maxEventBatchContentSize, + true + ) } assertEquals(exception.message, "'maxEventBatchContentSize' should be greater than zero, actual: $maxEventBatchContentSize") } @@ -51,7 +68,14 @@ class RuleConfigurationTest { val timeout = -1L val exception = assertThrows { - RuleConfiguration(TaskTimeout(timeout, 0), null, Duration.ofSeconds(1), 0.005, 1) + RuleConfiguration( + TaskTimeout(timeout, 0), + null, + Duration.ofSeconds(1), + 0.005, + 1, + true + ) } assertEquals(exception.message, "'timeout' should be set or be greater than zero, actual: $timeout") } diff --git a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt index 83a78979..695a89f3 100644 --- a/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt +++ b/src/test/kotlin/com/exactpro/th2/check1/rule/AbstractCheckTaskTest.kt @@ -157,7 +157,14 @@ abstract class AbstractCheckTaskTest { } protected fun createRuleConfiguration(taskTimeout: TaskTimeout, description: String = "Test", maxEventBatchContentSize: Int = 1024 * 1024): RuleConfiguration { - return RuleConfiguration(taskTimeout, description, configuration.timePrecision, configuration.decimalPrecision, maxEventBatchContentSize) + return RuleConfiguration( + taskTimeout, + description, + configuration.timePrecision, + configuration.decimalPrecision, + maxEventBatchContentSize, + true + ) } From b6fec84b44fc6b4e0f19b068ab76ec6ea2a5b38f Mon Sep 17 00:00:00 2001 From: Yaroslav Kuznetsov Date: Wed, 8 Dec 2021 12:42:27 +0400 Subject: [PATCH 41/41] [TH2-2773] Updated Check1 configuration (#121) * [TH2-2773] Updated Check1 configuration * [TH2-2773] Corrected README.md --- README.md | 4 ++-- .../th2/check1/configuration/Check1Configuration.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 971dae3e..56172bcf 100644 --- a/README.md +++ b/README.md @@ -122,7 +122,7 @@ This block describes the configuration for check1. "auto-silence-check-after-sequence-rule": false, "time-precision": "PT0.000000001S", "decimal-precision": 0.00001, - "check-null-value-as-empty": true + "check-null-value-as-empty": false } ``` @@ -154,7 +154,7 @@ The time precision is used to compare two time values. It is based on the `ISO-8 The decimal precision is used to compare the value of two numbers. Can be specified in number or string format. For example `0.0001`, `0.125`, `125E-3` #### check-null-value-as-empty -`check-null-value-as-empty` is used for `EMPTY` and `NOT_EMPTY` operations to check if `NULL_VALUE` value is empty. For example, if the `checkNullValueAsEmpty` parameter is: +`check-null-value-as-empty` is used for `EMPTY` and `NOT_EMPTY` operations to check if `NULL_VALUE` value is empty. By default, this parameter is set to `false`. For example, if the `checkNullValueAsEmpty` parameter is: + `true`, then `NULL_VALUE` is equal to `EMPTY`, otherwise `NULL_VALUE` is equal to `NOT_EMPTY` ## Required pins diff --git a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java index bf983bdc..469907b6 100644 --- a/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java +++ b/src/main/java/com/exactpro/th2/check1/configuration/Check1Configuration.java @@ -51,7 +51,7 @@ public class Check1Configuration { private Duration timePrecision = Duration.parse("PT0.000000001S"); @JsonProperty(value = "check-null-value-as-empty") - private boolean checkNullValueAsEmpty = true; + private boolean checkNullValueAsEmpty = false; public int getMessageCacheSize() { return messageCacheSize;