From 1a4926c4388f1b35d4cf562876b2e1c37d39a5de Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Thu, 2 Mar 2023 17:07:44 +0400 Subject: [PATCH 01/31] MessagesWorker > use separate executor for storeMessageBatch/storeGroupedMessageBatch --- .../cassandra/workers/MessagesWorker.java | 46 ++++++++++++++----- 1 file changed, 35 insertions(+), 11 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java index 9b0a77745..fdaa2599b 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java @@ -18,18 +18,40 @@ import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; import com.datastax.oss.driver.api.core.cql.Row; -import com.exactpro.cradle.*; +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.BookInfo; +import com.exactpro.cradle.Direction; +import com.exactpro.cradle.PageId; +import com.exactpro.cradle.PageInfo; +import com.exactpro.cradle.SessionRecordType; import com.exactpro.cradle.cassandra.counters.MessageStatisticsCollector; import com.exactpro.cradle.cassandra.counters.SessionStatisticsCollector; import com.exactpro.cradle.cassandra.dao.CassandraOperators; import com.exactpro.cradle.cassandra.dao.cache.CachedPageSession; import com.exactpro.cradle.cassandra.dao.cache.CachedSession; -import com.exactpro.cradle.cassandra.dao.messages.*; +import com.exactpro.cradle.cassandra.dao.messages.GroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchesIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.MessagesIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.PageGroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.PageSessionEntity; +import com.exactpro.cradle.cassandra.dao.messages.SessionEntity; import com.exactpro.cradle.cassandra.dao.messages.converters.MessageBatchEntityConverter; import com.exactpro.cradle.cassandra.resultset.CassandraCradleResultSet; import com.exactpro.cradle.cassandra.utils.GroupedMessageEntityUtils; import com.exactpro.cradle.cassandra.utils.MessageBatchEntityUtils; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.GroupedMessageFilter; +import com.exactpro.cradle.messages.MessageBatchToStore; +import com.exactpro.cradle.messages.MessageFilter; +import com.exactpro.cradle.messages.StoredGroupedMessageBatch; +import com.exactpro.cradle.messages.StoredMessage; +import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.resultset.CradleResultSet; import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.utils.CradleStorageException; @@ -51,7 +73,9 @@ import java.util.zip.DataFormatException; import static com.exactpro.cradle.CradleStorage.EMPTY_MESSAGE_INDEX; -import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.*; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_FIRST_MESSAGE_TIME; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_LAST_SEQUENCE; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_SEQUENCE; import static java.lang.String.format; public class MessagesWorker extends Worker @@ -328,7 +352,7 @@ public CompletableFuture storeMessageBatch(MessageBatchToStore batch, Page } catch (Exception e) { throw new CompletionException(e); } - }).thenComposeAsync(serializedEntity -> { + }, composingService).thenComposeAsync(serializedEntity -> { MessageBatchEntity entity = serializedEntity.getEntity(); List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); @@ -344,7 +368,7 @@ public CompletableFuture storeMessageBatch(MessageBatchToStore batch, Page entity.getSessionAlias(), meta), composingService) .thenRunAsync(() -> updateMessageWriteMetrics(entity, bookId), composingService); - }); + }, composingService); } public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToStore batchToStore, PageId pageId) { @@ -357,13 +381,13 @@ public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToSto } catch (Exception e) { throw new CompletionException(e); } - }).thenComposeAsync(serializedEntity -> { + }, composingService).thenComposeAsync(serializedEntity -> { GroupedMessageBatchEntity entity = serializedEntity.getEntity(); List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); return gmbOperator.write(entity, writeAttrs) - .thenComposeAsync((unused) -> storePageGroup(entity)) - .thenComposeAsync((unused) -> storeGroup(entity)) + .thenComposeAsync((unused) -> storePageGroup(entity), composingService) + .thenComposeAsync((unused) -> storeGroup(entity), composingService) .thenRunAsync(() -> messageStatisticsCollector.updateMessageBatchStatistics(bookId, pageId.getName(), entity.getGroup(), @@ -375,7 +399,7 @@ public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToSto entity.getGroup(), meta), composingService) .thenRunAsync(() -> updateMessageWriteMetrics(entity, bookId), composingService); - }); + }, composingService); } public long getBoundarySequence(String sessionAlias, Direction direction, BookInfo book, boolean first) @@ -416,4 +440,4 @@ public long getBoundarySequence(String sessionAlias, Direction direction, BookIn return row.getLong(first ? FIELD_SEQUENCE : FIELD_LAST_SEQUENCE); } -} \ No newline at end of file +} From 9930c3c5ee23d85ea50e1a24e8399ce1002b96af Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 6 Mar 2023 12:43:24 +0400 Subject: [PATCH 02/31] Added the composingServiceThreads setting --- .../cassandra/CassandraCradleStorage.java | 85 +++++++++---------- .../cassandra/CassandraStorageSettings.java | 36 +++++--- .../com/exactpro/cradle/CradleStorage.java | 41 +++++---- 3 files changed, 84 insertions(+), 78 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java index 646314141..0c3493753 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java @@ -1,17 +1,17 @@ /* - * Copyright 2020-2023 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle.cassandra; @@ -81,7 +81,7 @@ public class CassandraCradleStorage extends CradleStorage { - private Logger logger = LoggerFactory.getLogger(CassandraCradleStorage.class); + private final Logger LOGGER = LoggerFactory.getLogger(CassandraCradleStorage.class); private final CassandraConnection connection; private final CassandraStorageSettings settings; @@ -105,7 +105,7 @@ public class CassandraCradleStorage extends CradleStorage public CassandraCradleStorage(CassandraConnectionSettings connectionSettings, CassandraStorageSettings storageSettings, ExecutorService composingService) throws CradleStorageException { - super(composingService, storageSettings.getMaxMessageBatchSize(), storageSettings.getMaxTestEventBatchSize()); + super(composingService, storageSettings.getComposingServiceThreads(), storageSettings.getMaxMessageBatchSize(), storageSettings.getMaxTestEventBatchSize()); this.connection = new CassandraConnection(connectionSettings, storageSettings.getTimeout()); this.settings = storageSettings; @@ -140,7 +140,7 @@ protected void doInit(boolean prepareStorage) throws CradleStorageException if (prepareStorage) createStorage(); else - logger.info("Storage creation skipped"); + LOGGER.info("Storage creation skipped"); Duration timeout = Duration.ofMillis(settings.getTimeout()); @@ -185,7 +185,7 @@ protected void doDispose() throws CradleStorageException statisticsWorker.stop(); if (connection.isRunning()) { - logger.info("Disconnecting from Cassandra..."); + LOGGER.info("Disconnecting from Cassandra..."); try { connection.stop(); @@ -196,7 +196,7 @@ protected void doDispose() throws CradleStorageException } } else - logger.info("Already disconnected from Cassandra"); + LOGGER.info("Already disconnected from Cassandra"); } @Override @@ -274,8 +274,7 @@ protected Collection doLoadPages(BookId bookId) throws CradleStorageEx } @Override - protected void doRemovePage(PageInfo page) throws CradleStorageException - { + protected void doRemovePage(PageInfo page) { PageId pageId = page.getId(); removeSessionData(pageId); @@ -367,7 +366,7 @@ protected void doStoreTestEvent(TestEventToStore event, PageInfo page) throws IO } @Override - protected CompletableFuture doStoreTestEventAsync(TestEventToStore event, PageInfo page) throws IOException, CradleStorageException { + protected CompletableFuture doStoreTestEventAsync(TestEventToStore event, PageInfo page) { PageId pageId = page.getId(); return eventsWorker.storeEvent(event, pageId) @@ -536,7 +535,7 @@ protected long doGetFirstSequence(String sessionAlias, Direction direction, Book } @Override - protected Collection doGetSessionAliases(BookId bookId) throws IOException, CradleStorageException + protected Collection doGetSessionAliases(BookId bookId) throws CradleStorageException { MappedAsyncPagingIterable entities; String queryInfo = String.format("Getting session aliases for book '%s'", bookId); @@ -564,11 +563,11 @@ protected Collection doGetSessionAliases(BookId bookId) throws IOExcepti } @Override - protected Collection doGetGroups(BookId bookId) throws IOException, CradleStorageException { + protected Collection doGetGroups(BookId bookId) throws CradleStorageException { String queryInfo = String.format("Getting groups for book '%s'", bookId); - MappedAsyncPagingIterable entities = null; + MappedAsyncPagingIterable entities; try { var future = operators.getGroupsOperator().get(bookId.getName(), getReadAttrs()); @@ -607,14 +606,13 @@ protected StoredTestEvent doGetTestEvent(StoredTestEventId id, PageId pageId) th } @Override - protected CompletableFuture doGetTestEventAsync(StoredTestEventId id, PageId pageId) throws CradleStorageException - { + protected CompletableFuture doGetTestEventAsync(StoredTestEventId id, PageId pageId) { return eventsWorker.getTestEvent(id, pageId); } @Override - protected CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) throws CradleStorageException, IOException + protected CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) throws IOException { try { @@ -627,16 +625,13 @@ protected CradleResultSet doGetTestEvents(TestEventFilter filte } @Override - protected CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) - throws CradleStorageException - { + protected CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) { return eventsWorker.getTestEvents(filter, book); } @Override - protected Collection doGetScopes(BookId bookId) throws IOException, CradleStorageException - { + protected Collection doGetScopes(BookId bookId) throws IOException { MappedAsyncPagingIterable entities; String queryInfo = String.format("get scopes for book '%s'", bookId); try @@ -675,7 +670,7 @@ protected CompletableFuture> doGetMessageCounters interval.getStart().toString(), interval.getEnd().toString()); - logger.info("Getting {}", queryInfo); + LOGGER.info("Getting {}", queryInfo); MessageStatisticsIteratorProvider iteratorProvider = new MessageStatisticsIteratorProvider(queryInfo, operators, getBookCache().getBook(bookId), @@ -725,7 +720,7 @@ protected CompletableFuture> doGetCountersAsync(B interval.getStart().toString(), interval.getEnd().toString()); - logger.info("Getting {}", queryInfo); + LOGGER.info("Getting {}", queryInfo); EntityStatisticsIteratorProvider iteratorProvider = new EntityStatisticsIteratorProvider(queryInfo, @@ -746,7 +741,7 @@ protected CompletableFuture> doGetCountersAsync(B protected CradleResultSet doGetCounters(BookId bookId, EntityType entityType, FrameType frameType, - Interval interval) throws CradleStorageException, IOException { + Interval interval) throws IOException { String queryInfo = String.format("Counters for %s with frameType-%s from %s to %s", entityType.name(), frameType.name(), @@ -766,14 +761,14 @@ protected CradleResultSet doGetCounters(BookId bookId, protected CompletableFuture doGetMessageCountAsync(BookId bookId, String sessionAlias, Direction direction, - Interval interval) throws CradleStorageException { + Interval interval) { String queryInfo = String.format("Cumulative count for Messages with session_alias-%s, direction-%s from %s to %s", sessionAlias, direction, interval.getStart().toString(), interval.getEnd().toString()); - logger.info("Getting {}", queryInfo); + LOGGER.info("Getting {}", queryInfo); List slices = StorageUtils.sliceInterval(interval); @@ -791,7 +786,7 @@ protected CompletableFuture doGetMessageCountAsync(BookId bookId, sum = sum.incrementedBy(res.next().getCounter()); } } catch (CradleStorageException | IOException e) { - logger.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); + LOGGER.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); } } @@ -800,7 +795,7 @@ protected CompletableFuture doGetMessageCountAsync(BookId bookId, } @Override - protected Counter doGetMessageCount(BookId bookId, String sessionAlias, Direction direction, Interval interval) throws CradleStorageException, IOException { + protected Counter doGetMessageCount(BookId bookId, String sessionAlias, Direction direction, Interval interval) throws IOException { String queryInfo = String.format("Cumulative count for Messages with session_alias-%s, direction-%s from %s to %s", sessionAlias, direction, @@ -817,13 +812,13 @@ protected Counter doGetMessageCount(BookId bookId, String sessionAlias, Directio } @Override - protected CompletableFuture doGetCountAsync(BookId bookId, EntityType entityType, Interval interval) throws CradleStorageException { + protected CompletableFuture doGetCountAsync(BookId bookId, EntityType entityType, Interval interval) { String queryInfo = String.format("Cumulative count for %s with from %s to %s", entityType.name(), interval.getStart().toString(), interval.getEnd().toString()); - logger.info("Getting {}", queryInfo); + LOGGER.info("Getting {}", queryInfo); List slices = StorageUtils.sliceInterval(interval); @@ -841,7 +836,7 @@ protected CompletableFuture doGetCountAsync(BookId bookId, EntityType e sum = sum.incrementedBy(res.next().getCounter()); } } catch (CradleStorageException | IOException e) { - logger.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); + LOGGER.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); } } @@ -850,7 +845,7 @@ protected CompletableFuture doGetCountAsync(BookId bookId, EntityType e } @Override - protected Counter doGetCount(BookId bookId, EntityType entityType, Interval interval) throws CradleStorageException, IOException { + protected Counter doGetCount(BookId bookId, EntityType entityType, Interval interval) throws IOException { String queryInfo = String.format("Cumulative count for %s with from %s to %s", entityType.name(), interval.getStart().toString(), @@ -1034,11 +1029,11 @@ protected void connectToCassandra() throws CradleStorageException { if (!connection.isRunning()) { - logger.info("Connecting to Cassandra..."); + LOGGER.info("Connecting to Cassandra..."); try { connection.start(); - logger.info("Connected to Cassandra"); + LOGGER.info("Connected to Cassandra"); } catch (Exception e) { @@ -1046,7 +1041,7 @@ protected void connectToCassandra() throws CradleStorageException } } else - logger.info("Already connected to Cassandra"); + LOGGER.info("Already connected to Cassandra"); } protected CassandraOperators createOperators(CqlSession session, CassandraStorageSettings settings) @@ -1059,9 +1054,9 @@ protected void createStorage() throws CradleStorageException { try { - logger.info("Creating storage"); + LOGGER.info("Creating storage"); new CradleInfoKeyspaceCreator(exec, settings).createAll(); - logger.info("Storage creation finished"); + LOGGER.info("Storage creation finished"); } catch (IOException e) { diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index 3d127eb6f..c5806f53f 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -1,17 +1,17 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle.cassandra; @@ -21,6 +21,8 @@ import com.exactpro.cradle.cassandra.connection.NetworkTopologyStrategy; import com.exactpro.cradle.cassandra.retries.SelectExecutionPolicy; +import static com.exactpro.cradle.CradleStorage.DEFAULT_COMPOSING_SERVICE_THREADS; + public class CassandraStorageSettings { public static final String SCHEMA_VERSION = "5.3.0"; public static final long DEFAULT_TIMEOUT = 5000; @@ -52,7 +54,7 @@ public class CassandraStorageSettings { schemaVersion; private int keyspaceReplicationFactor; - private int maxParallelQueries, + private int maxParallelQueries, //FIXME: remove resultPageSize, maxMessageBatchSize, maxUncompressedMessageBatchSize, @@ -66,7 +68,8 @@ public class CassandraStorageSettings { pageGroupsCacheSize, groupsCacheSize, eventBatchDurationCacheSize, - counterPersistenceInterval; + counterPersistenceInterval, + composingServiceThreads; private SelectExecutionPolicy multiRowResultExecutionPolicy, singleRowResultExecutionPolicy; @@ -101,6 +104,7 @@ public CassandraStorageSettings(NetworkTopologyStrategy networkTopologyStrategy, this.scopesCacheSize = DEFAULT_SCOPES_CACHE_SIZE; this.pageScopesCacheSize = DEFAULT_PAGE_SCOPES_CACHE_SIZE; this.counterPersistenceInterval = DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS; + this.composingServiceThreads = DEFAULT_COMPOSING_SERVICE_THREADS; this.bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; this.sessionsCacheSize = DEFAULT_SESSIONS_CACHE_SIZE; this.sessionStatisticsCacheSize = DEFAULT_SESSION_STATISTICS_CACHE_SIZE; @@ -302,6 +306,14 @@ public void setCounterPersistenceInterval(int counterPersistenceInterval) { this.counterPersistenceInterval = counterPersistenceInterval; } + public int getComposingServiceThreads() { + return composingServiceThreads; + } + + public void setComposingServiceThreads(int composingServiceThreads) { + this.composingServiceThreads = composingServiceThreads; + } + public SelectExecutionPolicy getMultiRowResultExecutionPolicy() { return multiRowResultExecutionPolicy; } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index e69e1f30f..56d9e9fd1 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -1,17 +1,17 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle; @@ -50,7 +50,8 @@ public abstract class CradleStorage public static final ZoneOffset TIMEZONE_OFFSET = ZoneOffset.UTC; public static final long EMPTY_MESSAGE_INDEX = -1L; public static final int DEFAULT_MAX_MESSAGE_BATCH_SIZE = 1024*1024, - DEFAULT_MAX_TEST_EVENT_BATCH_SIZE = DEFAULT_MAX_MESSAGE_BATCH_SIZE; + DEFAULT_MAX_TEST_EVENT_BATCH_SIZE = DEFAULT_MAX_MESSAGE_BATCH_SIZE, + DEFAULT_COMPOSING_SERVICE_THREADS = 5; protected BookManager bookManager; private volatile boolean initialized = false, @@ -59,16 +60,14 @@ public abstract class CradleStorage protected final boolean ownedComposingService; protected final CradleEntitiesFactory entitiesFactory; - public CradleStorage(ExecutorService composingService, int maxMessageBatchSize, - int maxTestEventBatchSize) throws CradleStorageException + public CradleStorage(ExecutorService composingService, int composingServiceThreads, int maxMessageBatchSize, + int maxTestEventBatchSize) throws CradleStorageException { - if (composingService == null) - { + if (composingService == null) { ownedComposingService = true; - this.composingService = Executors.newFixedThreadPool(5); - } - else - { + this.composingService = Executors.newFixedThreadPool(composingServiceThreads); + logger.info("Created composing service executor with {} threads", composingServiceThreads); + } else { ownedComposingService = false; this.composingService = composingService; } @@ -78,8 +77,8 @@ public CradleStorage(ExecutorService composingService, int maxMessageBatchSize, public CradleStorage() throws CradleStorageException { - this(null, DEFAULT_MAX_MESSAGE_BATCH_SIZE, - DEFAULT_MAX_TEST_EVENT_BATCH_SIZE); + this(null, DEFAULT_COMPOSING_SERVICE_THREADS, + DEFAULT_MAX_MESSAGE_BATCH_SIZE, DEFAULT_MAX_TEST_EVENT_BATCH_SIZE); } From 6476db659e27f5fac5a66c8089141de526693170 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 6 Mar 2023 13:07:43 +0400 Subject: [PATCH 03/31] Corrected test --- .../com/exactpro/cradle/cassandra/CassandraStorageSettings.java | 1 + 1 file changed, 1 insertion(+) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index c5806f53f..111660019 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -141,6 +141,7 @@ public CassandraStorageSettings(CassandraStorageSettings settings) { this.groupsCacheSize = settings.getGroupsCacheSize(); this.sessionStatisticsCacheSize = settings.getSessionStatisticsCacheSize(); this.counterPersistenceInterval = settings.getCounterPersistenceInterval(); + this.composingServiceThreads = settings.getComposingServiceThreads(); this.bookRefreshIntervalMillis = settings.getBookRefreshIntervalMillis(); this.eventBatchDurationMillis = settings.getEventBatchDurationMillis(); this.eventBatchDurationCacheSize = settings.getEventBatchDurationCacheSize(); From fcc83181ebe08edf35e793a2c5da42d61a617419 Mon Sep 17 00:00:00 2001 From: georgiano <15014186+georgiano@users.noreply.github.com> Date: Mon, 6 Mar 2023 14:06:30 +0400 Subject: [PATCH 04/31] Future composition improvements (#221) * revised EventsWorker & MessagesWorker * storeGroupedMessageBatchAsync improvement --------- Co-authored-by: georgiano --- .../cassandra/workers/EventsWorker.java | 53 +++++---- .../cassandra/workers/MessagesWorker.java | 107 +++++++----------- .../com/exactpro/cradle/CradleStorage.java | 50 ++++---- 3 files changed, 101 insertions(+), 109 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java index 9c8f9107d..ab4346a4a 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -16,38 +16,40 @@ package com.exactpro.cradle.cassandra.workers; -import java.io.IOException; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.zip.DataFormatException; - -import com.exactpro.cradle.*; +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.BookInfo; +import com.exactpro.cradle.EntityType; +import com.exactpro.cradle.PageId; import com.exactpro.cradle.cassandra.EventBatchDurationWorker; import com.exactpro.cradle.cassandra.counters.BookStatisticsRecordsCaches; import com.exactpro.cradle.cassandra.counters.EntityStatisticsCollector; -import com.exactpro.cradle.cassandra.dao.testevents.*; -import com.exactpro.cradle.cassandra.dao.testevents.converters.TestEventEntityConverter; -import com.exactpro.cradle.serialization.SerializedEntityMetadata; -import com.exactpro.cradle.utils.CradleIdException; -import com.exactpro.cradle.utils.CradleStorageException; -import io.prometheus.client.Counter; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.exactpro.cradle.cassandra.dao.CassandraOperators; -import com.exactpro.cradle.cassandra.dao.cache.CachedScope; import com.exactpro.cradle.cassandra.dao.cache.CachedPageScope; +import com.exactpro.cradle.cassandra.dao.cache.CachedScope; +import com.exactpro.cradle.cassandra.dao.testevents.*; +import com.exactpro.cradle.cassandra.dao.testevents.converters.TestEventEntityConverter; import com.exactpro.cradle.cassandra.resultset.CassandraCradleResultSet; import com.exactpro.cradle.resultset.CradleResultSet; +import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.testevents.StoredTestEvent; import com.exactpro.cradle.testevents.StoredTestEventId; import com.exactpro.cradle.testevents.TestEventFilter; import com.exactpro.cradle.testevents.TestEventToStore; +import com.exactpro.cradle.utils.CradleIdException; +import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.TimeUtils; +import io.prometheus.client.Counter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.zip.DataFormatException; public class EventsWorker extends Worker { @@ -105,7 +107,7 @@ public CompletableFuture storeEvent(TestEventToStore event, PageId pageId) } catch (Exception e) { throw new CompletionException(e); } - }).thenComposeAsync(serializedEntity -> { + }, composingService).thenCompose(serializedEntity -> { TestEventEntity entity = serializedEntity.getEntity(); List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); @@ -130,9 +132,10 @@ public CompletableFuture storeEvent(TestEventToStore event, PageId pageId) } catch (CradleStorageException e) { logger.error("Exception while updating max duration {}", e.getMessage()); } - }) - .thenRunAsync(() -> entityStatisticsCollector.updateEntityBatchStatistics(pageId.getBookId(), key, meta), composingService) - .thenRunAsync(() -> updateEventWriteMetrics(entity, pageId.getBookId()), composingService); + + entityStatisticsCollector.updateEntityBatchStatistics(pageId.getBookId(), key, meta); + updateEventWriteMetrics(entity, pageId.getBookId()); + }, composingService); }); } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java index fdaa2599b..17371f927 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -18,40 +18,18 @@ import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; import com.datastax.oss.driver.api.core.cql.Row; -import com.exactpro.cradle.BookId; -import com.exactpro.cradle.BookInfo; -import com.exactpro.cradle.Direction; -import com.exactpro.cradle.PageId; -import com.exactpro.cradle.PageInfo; -import com.exactpro.cradle.SessionRecordType; +import com.exactpro.cradle.*; import com.exactpro.cradle.cassandra.counters.MessageStatisticsCollector; import com.exactpro.cradle.cassandra.counters.SessionStatisticsCollector; import com.exactpro.cradle.cassandra.dao.CassandraOperators; import com.exactpro.cradle.cassandra.dao.cache.CachedPageSession; import com.exactpro.cradle.cassandra.dao.cache.CachedSession; -import com.exactpro.cradle.cassandra.dao.messages.GroupEntity; -import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity; -import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchOperator; -import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageIteratorProvider; -import com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity; -import com.exactpro.cradle.cassandra.dao.messages.MessageBatchOperator; -import com.exactpro.cradle.cassandra.dao.messages.MessageBatchesIteratorProvider; -import com.exactpro.cradle.cassandra.dao.messages.MessagesIteratorProvider; -import com.exactpro.cradle.cassandra.dao.messages.PageGroupEntity; -import com.exactpro.cradle.cassandra.dao.messages.PageSessionEntity; -import com.exactpro.cradle.cassandra.dao.messages.SessionEntity; +import com.exactpro.cradle.cassandra.dao.messages.*; import com.exactpro.cradle.cassandra.dao.messages.converters.MessageBatchEntityConverter; import com.exactpro.cradle.cassandra.resultset.CassandraCradleResultSet; import com.exactpro.cradle.cassandra.utils.GroupedMessageEntityUtils; import com.exactpro.cradle.cassandra.utils.MessageBatchEntityUtils; -import com.exactpro.cradle.messages.GroupedMessageBatchToStore; -import com.exactpro.cradle.messages.GroupedMessageFilter; -import com.exactpro.cradle.messages.MessageBatchToStore; -import com.exactpro.cradle.messages.MessageFilter; -import com.exactpro.cradle.messages.StoredGroupedMessageBatch; -import com.exactpro.cradle.messages.StoredMessage; -import com.exactpro.cradle.messages.StoredMessageBatch; -import com.exactpro.cradle.messages.StoredMessageId; +import com.exactpro.cradle.messages.*; import com.exactpro.cradle.resultset.CradleResultSet; import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.utils.CradleStorageException; @@ -73,9 +51,7 @@ import java.util.zip.DataFormatException; import static com.exactpro.cradle.CradleStorage.EMPTY_MESSAGE_INDEX; -import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_FIRST_MESSAGE_TIME; -import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_LAST_SEQUENCE; -import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_SEQUENCE; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.*; import static java.lang.String.format; public class MessagesWorker extends Worker @@ -270,7 +246,7 @@ public CompletableFuture getMessage(StoredMessageId id, PageId pa }, composingService); } - public CompletableFuture storePageGroup (GroupedMessageBatchEntity groupedMessageBatchEntity) { + private CompletableFuture storePageGroup (GroupedMessageBatchEntity groupedMessageBatchEntity) { CassandraOperators operators = getOperators(); PageGroupEntity pageGroupEntity = new PageGroupEntity( groupedMessageBatchEntity.getBook(), @@ -284,14 +260,14 @@ public CompletableFuture storePageGroup (GroupedMessageBatchEnt logger.debug("Writing group '{}' for page '{}'", pageGroupEntity.getGroup(), pageGroupEntity.getPage()); return operators.getPageGroupsOperator().write(pageGroupEntity, writeAttrs) - .whenComplete((result, e) -> { + .whenCompleteAsync((result, e) -> { if (e == null) { operators.getPageGroupCache().store(pageGroupEntity); } - }); + }, composingService); } - public CompletableFuture storeGroup (GroupedMessageBatchEntity groupedMessageBatchEntity) { + private CompletableFuture storeGroup (GroupedMessageBatchEntity groupedMessageBatchEntity) { CassandraOperators operators = getOperators(); GroupEntity groupEntity = new GroupEntity(groupedMessageBatchEntity.getBook(), groupedMessageBatchEntity.getGroup()); @@ -302,11 +278,11 @@ public CompletableFuture storeGroup (GroupedMessageBatchEntity grou logger.debug("Writing group '{}'", groupEntity.getGroup()); return operators.getGroupsOperator().write(groupEntity, writeAttrs) - .whenComplete((result, e) -> { + .whenCompleteAsync((result, e) -> { if (e == null) { operators.getGroupCache().store(groupEntity); } - }); + }, composingService); } public CompletableFuture storePageSession(MessageBatchToStore batch, PageId pageId) @@ -352,23 +328,25 @@ public CompletableFuture storeMessageBatch(MessageBatchToStore batch, Page } catch (Exception e) { throw new CompletionException(e); } - }, composingService).thenComposeAsync(serializedEntity -> { + }, composingService).thenCompose(serializedEntity -> { MessageBatchEntity entity = serializedEntity.getEntity(); List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); return mbOperator.write(entity, writeAttrs) - .thenRunAsync(() -> messageStatisticsCollector.updateMessageBatchStatistics(bookId, - entity.getPage(), - entity.getSessionAlias(), - entity.getDirection(), - meta), composingService) - .thenRunAsync(() -> sessionStatisticsCollector.updateSessionStatistics(bookId, - entity.getPage(), - SessionRecordType.SESSION, - entity.getSessionAlias(), - meta), composingService) - .thenRunAsync(() -> updateMessageWriteMetrics(entity, bookId), composingService); - }, composingService); + .thenRunAsync(() -> { + messageStatisticsCollector.updateMessageBatchStatistics(bookId, + entity.getPage(), + entity.getSessionAlias(), + entity.getDirection(), + meta); + sessionStatisticsCollector.updateSessionStatistics(bookId, + entity.getPage(), + SessionRecordType.SESSION, + entity.getSessionAlias(), + meta); + updateMessageWriteMetrics(entity, bookId); + }, composingService); + }); } public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToStore batchToStore, PageId pageId) { @@ -381,27 +359,30 @@ public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToSto } catch (Exception e) { throw new CompletionException(e); } - }, composingService).thenComposeAsync(serializedEntity -> { + }, composingService).thenCompose(serializedEntity -> { GroupedMessageBatchEntity entity = serializedEntity.getEntity(); List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); return gmbOperator.write(entity, writeAttrs) - .thenComposeAsync((unused) -> storePageGroup(entity), composingService) - .thenComposeAsync((unused) -> storeGroup(entity), composingService) - .thenRunAsync(() -> messageStatisticsCollector.updateMessageBatchStatistics(bookId, - pageId.getName(), - entity.getGroup(), - "", - meta), composingService) - .thenRunAsync(() -> sessionStatisticsCollector.updateSessionStatistics(bookId, - pageId.getName(), - SessionRecordType.SESSION_GROUP, - entity.getGroup(), - meta), composingService) - .thenRunAsync(() -> updateMessageWriteMetrics(entity, bookId), composingService); - }, composingService); + .thenRunAsync(() -> storePageGroup(entity), composingService) + .thenRunAsync(() -> storeGroup(entity), composingService) + .thenRunAsync(() -> { + messageStatisticsCollector.updateMessageBatchStatistics(bookId, + pageId.getName(), + entity.getGroup(), + "", + meta); + sessionStatisticsCollector.updateSessionStatistics(bookId, + pageId.getName(), + SessionRecordType.SESSION_GROUP, + entity.getGroup(), + meta); + updateMessageWriteMetrics(entity, bookId); + }, composingService); + }); } + public long getBoundarySequence(String sessionAlias, Direction direction, BookInfo book, boolean first) throws CradleStorageException { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index 56d9e9fd1..7ec560bd1 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -1,17 +1,17 @@ /* - * Copyright 2023 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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 + * 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 + * 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. + * 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.cradle; @@ -38,6 +38,7 @@ import java.time.ZoneOffset; import java.util.*; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -582,20 +583,27 @@ public final CompletableFuture storeGroupedMessageBatchAsync(GroupedMessag String id = String.format("%s:%s", batch.getBookId(), batch.getFirstTimestamp().toString()); logger.debug("Storing message batch {} grouped by {} asynchronously", id, groupName); - var batches = paginateBatch(batch); - - CompletableFuture[] futures = new CompletableFuture[batches.size()]; - int i = 0; - for (var b : batches) { - CompletableFuture future; + CompletableFuture result = CompletableFuture.supplyAsync(() -> { try { - future = doStoreGroupedMessageBatchAsync(b.getKey(), b.getValue()); + return paginateBatch(batch); } catch (Exception e) { - future = CompletableFuture.failedFuture(e); + throw new CompletionException(e); } - futures[i++] = future; - } - CompletableFuture result = CompletableFuture.allOf(futures); + }, composingService).thenCompose((batches) -> { + CompletableFuture[] futures = new CompletableFuture[batches.size()]; + int i = 0; + for (var b : batches) { + CompletableFuture future; + try { + future = doStoreGroupedMessageBatchAsync(b.getKey(), b.getValue()); + } catch (Exception e) { + future = CompletableFuture.failedFuture(e); + } + futures[i++] = future; + } + return CompletableFuture.allOf(futures); + }); + result.whenCompleteAsync((r, error) -> { if (error != null) logger.error("Error while storing message batch "+id+" grouped by "+groupName+" asynchronously", error); From e0e6fe6c4a3388149e9d963b9d537d9085a830a5 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 6 Mar 2023 15:32:22 +0400 Subject: [PATCH 05/31] Use thread factory for executors --- .../cassandra/workers/StatisticsWorker.java | 26 +++++++++++-------- .../java/com/exactpro/cradle/BookManager.java | 26 +++++++++++-------- .../com/exactpro/cradle/CradleStorage.java | 26 +++++++++++-------- 3 files changed, 45 insertions(+), 33 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java index 863b9053e..3f25d9425 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java @@ -1,22 +1,23 @@ /* - * Copyright 2020-2023 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle.cassandra.workers; import com.datastax.oss.driver.api.core.cql.AsyncResultSet; import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder; +import com.datastax.oss.driver.shaded.guava.common.util.concurrent.ThreadFactoryBuilder; import com.exactpro.cradle.BookId; import com.exactpro.cradle.EntityType; import com.exactpro.cradle.FrameType; @@ -44,6 +45,8 @@ public class StatisticsWorker implements Runnable, EntityStatisticsCollector, Me private static final Logger logger = LoggerFactory.getLogger(StatisticsWorker.class); private static final int MAX_BATCH_STATEMENTS = 16384; + private static final ThreadFactory THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("cradle-statistics-worker-%d").build(); + private final FutureTracker futures; private final CassandraOperators operators; private final long interval; @@ -58,6 +61,7 @@ public StatisticsWorker (WorkerSupplies workerSupplies, long persistenceInterval this.interval = persistenceInterval; this.bookCounterCaches = new ConcurrentHashMap<>(); this.isEnabled = (interval != 0); + logger.info("Statistics worker status is {}", this.isEnabled); } private ScheduledExecutorService executorService; @@ -68,7 +72,7 @@ public void start() { } logger.info("Starting executor for StatisticsWorker"); - executorService = Executors.newScheduledThreadPool(1); + executorService = Executors.newScheduledThreadPool(1, THREAD_FACTORY); executorService.scheduleAtFixedRate(this, 0, interval, TimeUnit.MILLISECONDS); logger.info("StatisticsWorker executor started"); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/BookManager.java b/cradle-core/src/main/java/com/exactpro/cradle/BookManager.java index da02b4024..946dbffca 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/BookManager.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/BookManager.java @@ -1,17 +1,17 @@ /* - * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle; @@ -19,15 +19,19 @@ import java.util.*; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import com.exactpro.cradle.utils.CradleStorageException; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class BookManager { private final static Logger logger = LoggerFactory.getLogger(BookManager.class); + + private static final ThreadFactory THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("cradle-book-manager-%d").build(); private final BookCache bookCache; private ScheduledExecutorService executorService; private Long refreshIntervalMillis; @@ -47,7 +51,7 @@ public void start() { return; } - executorService = Executors.newScheduledThreadPool(1); + executorService = Executors.newScheduledThreadPool(1, THREAD_FACTORY); logger.debug("Registered refresher task for cached books"); executorService.scheduleWithFixedDelay(new Refresher(bookCache), refreshIntervalMillis, refreshIntervalMillis, TimeUnit.MILLISECONDS); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index 7ec560bd1..3a32f0f5e 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -1,17 +1,17 @@ /* - * Copyright 2020-2023 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle; @@ -29,6 +29,7 @@ import com.exactpro.cradle.utils.BookPagesNamesChecker; import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.TestEventUtils; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,6 +42,7 @@ import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; /** * Storage which holds information about all data sent or received and test events. @@ -54,6 +56,8 @@ public abstract class CradleStorage DEFAULT_MAX_TEST_EVENT_BATCH_SIZE = DEFAULT_MAX_MESSAGE_BATCH_SIZE, DEFAULT_COMPOSING_SERVICE_THREADS = 5; + private static final ThreadFactory THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("cradle-storage-%d").build(); + protected BookManager bookManager; private volatile boolean initialized = false, disposed = false; @@ -66,7 +70,7 @@ public CradleStorage(ExecutorService composingService, int composingServiceThrea { if (composingService == null) { ownedComposingService = true; - this.composingService = Executors.newFixedThreadPool(composingServiceThreads); + this.composingService = Executors.newFixedThreadPool(composingServiceThreads, THREAD_FACTORY); logger.info("Created composing service executor with {} threads", composingServiceThreads); } else { ownedComposingService = false; From 568428d62e636cb14b74bcca0dc72cbf0a2742bb Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 6 Mar 2023 18:45:11 +0400 Subject: [PATCH 06/31] Refactored StoredMessage comparator --- .../CounterTimeFrameRecordFactory.java | 15 ++++++ .../SessionsTimeFrameRecordFactory.java | 15 ++++++ .../cassandra/counters/TimeFrameRecord.java | 15 ++++++ .../counters/TimeFrameRecordFactory.java | 15 ++++++ .../messages/StoredGroupedMessageBatch.java | 49 ++++++------------- 5 files changed, 76 insertions(+), 33 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/CounterTimeFrameRecordFactory.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/CounterTimeFrameRecordFactory.java index 145258780..1b8741b2f 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/CounterTimeFrameRecordFactory.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/CounterTimeFrameRecordFactory.java @@ -1,3 +1,18 @@ +/* + * Copyright 2023 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.cradle.cassandra.counters; import com.exactpro.cradle.counters.Counter; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionsTimeFrameRecordFactory.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionsTimeFrameRecordFactory.java index 62c969938..6e5d09f63 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionsTimeFrameRecordFactory.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionsTimeFrameRecordFactory.java @@ -1,3 +1,18 @@ +/* + * Copyright 2023 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.cradle.cassandra.counters; import java.time.Instant; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecord.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecord.java index d7943796d..4ca5d016c 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecord.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecord.java @@ -1,3 +1,18 @@ +/* + * Copyright 2023 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.cradle.cassandra.counters; import java.time.Instant; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecordFactory.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecordFactory.java index fb98e81b9..d8aec39af 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecordFactory.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/TimeFrameRecordFactory.java @@ -1,3 +1,18 @@ +/* + * Copyright 2023 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.cradle.cassandra.counters; import java.time.Instant; diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java index 5b36f98f2..3324fd98a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java @@ -1,17 +1,17 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle.messages; @@ -98,33 +98,16 @@ public boolean isEmpty() { return messages.isEmpty(); } - static class StoredMessageComparator implements Comparator { - - @Override - public int compare(StoredMessage o1, StoredMessage o2) { - int r; - - r = Comparator.comparing(StoredMessage::getTimestamp).compare(o1, o2); - if (r != 0) - return r; - - r = Comparator.comparing(StoredMessage::getSessionAlias).compare(o1, o2); - if (r != 0) - return r; - - r = Comparator.comparing(StoredMessage::getDirection).compare(o1, o2); - if (r != 0) - return r; - - return Comparator.comparingLong(StoredMessage::getSequence).compare(o1, o2); - } - } + public static final Comparator STORED_MESSAGE_COMPARATOR = Comparator.comparing(StoredMessage::getTimestamp) + .thenComparing(StoredMessage::getSessionAlias) + .thenComparing(StoredMessage::getDirection) + .thenComparingLong(StoredMessage::getSequence); protected TreeSet createMessagesCollection(Collection messages, PageId pageId) { if (messages == null) - return new TreeSet<>(new StoredMessageComparator()); + return new TreeSet<>(STORED_MESSAGE_COMPARATOR); - TreeSet result = new TreeSet<>(new StoredMessageComparator()); + TreeSet result = new TreeSet<>(STORED_MESSAGE_COMPARATOR); for (StoredMessage msg : messages) result.add(new StoredMessage(msg, msg.getId(), pageId)); return result; From 08f6fb85ffc5f30c8e269f27108f519f6246dba3 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 13 Mar 2023 10:13:54 +0400 Subject: [PATCH 07/31] Added FIXMEs afret profiling --- .../src/main/java/com/exactpro/cradle/CradleStorage.java | 1 + .../exactpro/cradle/messages/GroupedMessageBatchToStore.java | 2 ++ .../com/exactpro/cradle/serialization/MessageSerializer.java | 2 ++ .../exactpro/cradle/serialization/MessagesSizeCalculator.java | 1 + 4 files changed, 6 insertions(+) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index 3a32f0f5e..cd8298e38 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -527,6 +527,7 @@ public final CompletableFuture storeMessageBatchAsync(MessageBatchToStore } + //FIXME: com.exactpro.cradle.CradleStorage.paginateBatch() 9,932 ms (6.5%) List> paginateBatch(GroupedMessageBatchToStore batch) throws CradleStorageException { BookId bookId = batch.getBookId(); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java index 1d1718cd5..920a780ce 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java @@ -81,6 +81,7 @@ private boolean hasSpace(int messageSize) { * @return immutable message object with assigned ID * @throws CradleStorageException if message cannot be added to the batch due to verification failure */ + //FIXME: java.util.TreeSet.add() 68,759 ms (28.3%) jmh test public StoredMessage addMessage(MessageToStore message) throws CradleStorageException { int expMsgSize = MessagesSizeCalculator.calculateMessageSizeInBatch(message); if (!hasSpace(expMsgSize)) @@ -201,6 +202,7 @@ public boolean addBatch(GroupedMessageBatchToStore batch) throws CradleStorageEx return true; } + //FIXME: com.exactpro.cradle.messages.GroupedMessageBatchToStore.getSessionMessageBatches() 24,485 ms (16.1%) public Collection getSessionMessageBatches() throws CradleStorageException{ Map batches = new HashMap<>(); for (StoredMessage message: getMessages()) { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java index 4f07652fa..369c15160 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java @@ -31,6 +31,7 @@ public class MessageSerializer { + //FIXME: com.exactpro.cradle.serialization.MessageSerializer.serializeBatch() 31,319 ms (20.6%) public SerializedEntityData serializeBatch(Collection batch) throws SerializationException { SerializationBatchSizes messageBatchSizes = MessagesSizeCalculator.calculateMessageBatchSize(batch); ByteBuffer buffer = ByteBuffer.allocate(messageBatchSizes.total); @@ -40,6 +41,7 @@ public SerializedEntityData serializeBatch(Collection batch) thro return new SerializedEntityData(serializedMessageMetadata, buffer.array()); } + //FIXME: com.exactpro.cradle.serialization.MessageSerializer.serializeBatch() 15,597 ms (10.3%) public List serializeBatch( Collection batch, ByteBuffer buffer, SerializationBatchSizes messageBatchSizes ) throws SerializationException { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java index 40c604e19..92a4610c1 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java @@ -74,6 +74,7 @@ public static int calculateStringSize(String str) { return (str != null ? str.getBytes(StandardCharsets.UTF_8).length : 0) + 4; } + //FIXME: com.exactpro.cradle.serialization.MessagesSizeCalculator.calculateMessageBatchSize() 12,657 ms (8.3%) public static SerializationBatchSizes calculateMessageBatchSize(Collection message) { SerializationBatchSizes sizes = new SerializationBatchSizes(message.size()); From d4358a949c9f35302c33538700240cf912f21ace Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Mon, 13 Mar 2023 13:21:40 +0400 Subject: [PATCH 08/31] StoredGroupedMessageBatch > use list instead of set to store messages --- .../messages/StoredGroupedMessageBatch.java | 32 +++++++++---------- .../GroupedMessageBatchToStoreTest.java | 32 ------------------- 2 files changed, 15 insertions(+), 49 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java index 3324fd98a..96dca1532 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java @@ -19,18 +19,21 @@ import com.exactpro.cradle.BookId; import com.exactpro.cradle.PageId; import com.exactpro.cradle.serialization.MessagesSizeCalculator; +import com.google.common.collect.Lists; import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; -import java.util.TreeSet; +import java.util.List; + +import static com.exactpro.cradle.serialization.MessagesSizeCalculator.MESSAGE_BATCH_CONST_VALUE; public class StoredGroupedMessageBatch { protected BookId bookId; private final String group; protected int batchSize; - protected final TreeSet messages; + protected final List messages; private final Instant recDate; public BookId getBookId() { @@ -46,7 +49,7 @@ public StoredGroupedMessageBatch(String group, Collection message this.group = group; this.messages = createMessagesCollection(messages, pageId); if (messages == null || messages.isEmpty()) { - batchSize = MessagesSizeCalculator.calculateMessageBatchSize(Collections.emptyList()).total; + batchSize = MESSAGE_BATCH_CONST_VALUE; return; } batchSize = MessagesSizeCalculator.calculateMessageBatchSize(messages).total; @@ -68,15 +71,15 @@ public Collection getMessages() { } public Collection getMessagesReverse() { - return Collections.unmodifiableCollection(messages.descendingSet()); + return Collections.unmodifiableCollection(Lists.reverse(messages)); } public StoredMessage getFirstMessage() { - return !messages.isEmpty() ? messages.first() : null; + return !messages.isEmpty() ? messages.get(0) : null; } public StoredMessage getLastMessage() { - return !messages.isEmpty() ? messages.last() : null; + return !messages.isEmpty() ? messages.get(messages.size() - 1) : null; } public Instant getFirstTimestamp() { @@ -98,18 +101,13 @@ public boolean isEmpty() { return messages.isEmpty(); } - public static final Comparator STORED_MESSAGE_COMPARATOR = Comparator.comparing(StoredMessage::getTimestamp) - .thenComparing(StoredMessage::getSessionAlias) - .thenComparing(StoredMessage::getDirection) - .thenComparingLong(StoredMessage::getSequence); - - protected TreeSet createMessagesCollection(Collection messages, PageId pageId) { + protected List createMessagesCollection(Collection messages, PageId pageId) { if (messages == null) - return new TreeSet<>(STORED_MESSAGE_COMPARATOR); + return new ArrayList<>(); - TreeSet result = new TreeSet<>(STORED_MESSAGE_COMPARATOR); + List result = new ArrayList<>(messages.size()); for (StoredMessage msg : messages) - result.add(new StoredMessage(msg, msg.getId(), pageId)); + result.add(new StoredMessage(msg, msg.getId(), pageId)); // why do we copy messages here? return result; } -} \ No newline at end of file +} diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java index 8ddbb71ed..3ca65dda9 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java @@ -321,39 +321,7 @@ public void correctMessageId() throws CradleStorageException Assert.assertEquals(msg.getId(), new StoredMessageId(book, sessionAlias, direction, timestamp, seq+1)); } - @Test - public void batchShowsLastTimestamp() throws CradleStorageException - { - Instant timestamp = Instant.ofEpochSecond(1000); - GroupedMessageBatchToStore batch = new GroupedMessageBatchToStore(groupName, MAX_SIZE); - batch.addMessage(builder - .bookId(book) - .sessionAlias(sessionAlias) - .direction(Direction.FIRST) - .sequence(1) - .timestamp(timestamp.plusSeconds(20)) - .content(messageContent) - .build()); - - batch.addMessage(builder - .bookId(book) - .sessionAlias(sessionAlias) - .direction(Direction.SECOND) - .timestamp(timestamp.plusSeconds(10)) - .content(messageContent) - .build()); - - batch.addMessage(builder - .bookId(book) - .sessionAlias(sessionAlias) - .direction(Direction.SECOND) - .timestamp(timestamp.plusSeconds(15)) - .content(messageContent) - .build()); - Assert.assertEquals(batch.getLastTimestamp(), timestamp.plusSeconds(20), "Last timestamp is incorrect"); - } - @Test public void batchSerialization() throws CradleStorageException, IOException { From 9b3bbf504f8241d6a40230381201b967bc6bc904 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 13 Mar 2023 19:22:09 +0400 Subject: [PATCH 09/31] Refactored metrics * added * cradle_test_events_stored_uncompressed_bytes_total * cradle_test_events_stored_compressed_bytes_total * cradle_message_stored_uncompressed_bytes_total * cradle_message_stored_compressed_bytes_total --- .../cassandra/workers/EventsWorker.java | 16 ++++++--- .../cassandra/workers/MessagesWorker.java | 33 ++++++++++++++----- 2 files changed, 37 insertions(+), 12 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java index ab4346a4a..d6de66066 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java @@ -55,10 +55,14 @@ public class EventsWorker extends Worker { private static final Logger logger = LoggerFactory.getLogger(EventsWorker.class); - private static final Counter EVENTS_READ_METRIC = Counter.build().name("cradle_test_events_readed") + private static final Counter EVENTS_READ_METRIC = Counter.build().name("cradle_test_events_read_total") .help("Fetched test events").labelNames(BOOK_ID, SCOPE).register(); - private static final Counter EVENTS_STORE_METRIC = Counter.build().name("cradle_test_events_stored") + private static final Counter EVENTS_STORE_METRIC = Counter.build().name("cradle_test_events_stored_total") .help("Stored test events").labelNames(BOOK_ID, SCOPE).register(); + private static final Counter EVENTS_STORE_UNCOMPRESSED_BYTES = Counter.build().name("cradle_test_events_stored_uncompressed_bytes_total") + .help("Stored uncompressed event bytes").labelNames(BOOK_ID, SCOPE).register(); + private static final Counter EVENTS_STORE_COMPRESSED_BYTES = Counter.build().name("cradle_test_events_stored_compressed_bytes_total") + .help("Stored compressed event bytes").labelNames(BOOK_ID, SCOPE).register(); private final EntityStatisticsCollector entityStatisticsCollector; private final EventBatchDurationWorker durationWorker; @@ -90,10 +94,14 @@ private static void updateEventReadMetrics(StoredTestEvent testEvent) .inc(testEvent.isSingle() ? 1 : testEvent.asBatch().getTestEventsCount()); } - private static void updateEventWriteMetrics(TestEventEntity entity, BookId bookId) - { + private static void updateEventWriteMetrics(TestEventEntity entity, BookId bookId) { EVENTS_STORE_METRIC.labels(bookId.getName(), entity.getScope()) .inc(entity.isEventBatch() ? entity.getEventCount() : 1); + + EVENTS_STORE_UNCOMPRESSED_BYTES.labels(bookId.getName(), entity.getScope()) + .inc(entity.getUncompressedContentSize()); + EVENTS_STORE_COMPRESSED_BYTES.labels(bookId.getName(), entity.getScope()) + .inc(entity.getContentSize()); } public CompletableFuture storeEvent(TestEventToStore event, PageId pageId) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java index 17371f927..3657b9bdb 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java @@ -58,10 +58,14 @@ public class MessagesWorker extends Worker { private static final Logger logger = LoggerFactory.getLogger(MessagesWorker.class); - private static final Counter MESSAGE_READ_METRIC = Counter.build().name("cradle_message_readed") + private static final Counter MESSAGE_READ_METRIC = Counter.build().name("cradle_message_read_total") .help("Fetched messages").labelNames(BOOK_ID, SESSION_ALIAS, DIRECTION).register(); - private static final Counter MESSAGE_WRITE_METRIC = Counter.build().name("cradle_message_stored") + private static final Counter MESSAGE_STORE_METRIC = Counter.build().name("cradle_message_stored_total") .help("Stored messages").labelNames(BOOK_ID, SESSION_ALIAS, DIRECTION).register(); + private static final Counter MESSAGE_STORE_UNCOMPRESSED_BYTES = Counter.build().name("cradle_message_stored_uncompressed_bytes_total") + .help("Stored uncompressed message bytes").labelNames(BOOK_ID, SESSION_ALIAS).register(); + private static final Counter MESSAGE_STORE_COMPRESSED_BYTES = Counter.build().name("cradle_message_stored_compressed_bytes_total") + .help("Stored compressed message bytes").labelNames(BOOK_ID, SESSION_ALIAS).register(); private final MessageStatisticsCollector messageStatisticsCollector; private final SessionStatisticsCollector sessionStatisticsCollector; @@ -116,18 +120,31 @@ private static void updateMessageReadMetrics(BookId bookId, StoredGroupedMessage .inc(batch.getMessageCount()); } - private static void updateMessageWriteMetrics(MessageBatchEntity entity, BookId bookId) - { - MESSAGE_WRITE_METRIC + private static void updateMessageWriteMetrics(MessageBatchEntity entity, BookId bookId) { + MESSAGE_STORE_METRIC .labels(bookId.getName(), entity.getSessionAlias(), entity.getDirection()) .inc(entity.getMessageCount()); + + MESSAGE_STORE_UNCOMPRESSED_BYTES + .labels(bookId.getName(), entity.getSessionAlias()) + .inc(entity.getUncompressedContentSize()); + MESSAGE_STORE_COMPRESSED_BYTES + .labels(bookId.getName(), entity.getSessionAlias()) + .inc(entity.getContentSize()); + } - private static void updateMessageWriteMetrics(GroupedMessageBatchEntity entity, BookId bookId) - { - MESSAGE_WRITE_METRIC + private static void updateMessageWriteMetrics(GroupedMessageBatchEntity entity, BookId bookId) { + MESSAGE_STORE_METRIC .labels(bookId.getName(), entity.getGroup(), "") .inc(entity.getMessageCount()); + + MESSAGE_STORE_UNCOMPRESSED_BYTES + .labels(bookId.getName(), entity.getGroup()) + .inc(entity.getUncompressedContentSize()); + MESSAGE_STORE_COMPRESSED_BYTES + .labels(bookId.getName(), entity.getGroup()) + .inc(entity.getContentSize()); } public CompletableFuture> getMessageBatches(MessageFilter filter, BookInfo book) From c02b72e391adb4694ae9549c7a23baeace266232 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Tue, 14 Mar 2023 14:20:35 +0400 Subject: [PATCH 10/31] Added storeIndividualMessageSessions option --- .../cassandra/CassandraCradleStorage.java | 23 +++++++++++-------- .../cassandra/CassandraStorageSettings.java | 13 +++++++++++ 2 files changed, 26 insertions(+), 10 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java index 0c3493753..e16a9c9a3 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java @@ -311,10 +311,12 @@ protected void doStoreGroupedMessageBatch(GroupedMessageBatchToStore batch, Page { messagesWorker.storeGroupedMessageBatch(batch, pageId).get(); - for (MessageBatchToStore b: batch.getSessionMessageBatches()) { - messagesWorker.storeMessageBatch(b, pageId).get(); - messagesWorker.storeSession(b).get(); - messagesWorker.storePageSession(b, pageId).get(); + if (settings.isStoreIndividualMessageSessions()) { + for (MessageBatchToStore b : batch.getSessionMessageBatches()) { + messagesWorker.storeMessageBatch(b, pageId).get(); + messagesWorker.storeSession(b).get(); + messagesWorker.storePageSession(b, pageId).get(); + } } } catch (Exception e) @@ -340,12 +342,13 @@ protected CompletableFuture doStoreGroupedMessageBatchAsync(GroupedMessage CompletableFuture future = messagesWorker.storeGroupedMessageBatch(batch, pageId); - // store individual session message batches - for (MessageBatchToStore b: batch.getSessionMessageBatches()) { - future = future.thenComposeAsync((unused) -> messagesWorker.storeMessageBatch(b, pageId), composingService) - .thenComposeAsync((unused) -> messagesWorker.storeSession(b), composingService) - .thenComposeAsync((unused) -> messagesWorker.storePageSession(b, pageId), composingService) - .thenAccept(NOOP); + if (settings.isStoreIndividualMessageSessions()) { + for (MessageBatchToStore b : batch.getSessionMessageBatches()) { + future = future.thenComposeAsync((unused) -> messagesWorker.storeMessageBatch(b, pageId), composingService) + .thenComposeAsync((unused) -> messagesWorker.storeSession(b), composingService) + .thenComposeAsync((unused) -> messagesWorker.storePageSession(b, pageId), composingService) + .thenAccept(NOOP); + } } return future; } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index 111660019..7483f46db 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -43,6 +43,7 @@ public class CassandraStorageSettings { DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS = 1000; public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000, DEFAULT_EVENT_BATCH_DURATION_MILLIS = 5_000; + public static final boolean DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS = true; @@ -75,6 +76,8 @@ public class CassandraStorageSettings { private long bookRefreshIntervalMillis, eventBatchDurationMillis; + private boolean storeIndividualMessageSessions; + public CassandraStorageSettings() { this(null, DEFAULT_TIMEOUT, DEFAULT_CONSISTENCY_LEVEL, DEFAULT_CONSISTENCY_LEVEL); } @@ -112,6 +115,7 @@ public CassandraStorageSettings(NetworkTopologyStrategy networkTopologyStrategy, this.groupsCacheSize = DEFAULT_GROUPS_CACHE_SIZE; this.eventBatchDurationCacheSize = DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE; this.eventBatchDurationMillis = DEFAULT_EVENT_BATCH_DURATION_MILLIS; + this.storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; } public CassandraStorageSettings(CassandraStorageSettings settings) { @@ -145,6 +149,8 @@ public CassandraStorageSettings(CassandraStorageSettings settings) { this.bookRefreshIntervalMillis = settings.getBookRefreshIntervalMillis(); this.eventBatchDurationMillis = settings.getEventBatchDurationMillis(); this.eventBatchDurationCacheSize = settings.getEventBatchDurationCacheSize(); + + this.storeIndividualMessageSessions = settings.isStoreIndividualMessageSessions(); } @@ -357,4 +363,11 @@ public void setEventBatchDurationCacheSize(int eventBatchDurationCacheSize) { this.eventBatchDurationCacheSize = eventBatchDurationCacheSize; } + public boolean isStoreIndividualMessageSessions() { + return storeIndividualMessageSessions; + } + + public void setStoreIndividualMessageSessions(boolean storeIndividualMessageSessions) { + this.storeIndividualMessageSessions = storeIndividualMessageSessions; + } } From d3e03333369c73b90d936f4660b2bf9fb9eec620 Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Thu, 16 Mar 2023 16:30:12 +0400 Subject: [PATCH 11/31] Serialization > calculate batch/message size as few times as possible --- .../utils/GroupedMessageEntityUtils.java | 2 +- .../utils/MessageBatchEntityUtils.java | 7 +++- .../cradle/messages/CradleMessage.java | 8 ++++ .../messages/GroupedMessageBatchToStore.java | 6 +-- .../cradle/messages/MessageBatchToStore.java | 13 +++--- .../cradle/messages/MessageToStore.java | 22 ++++++++-- .../messages/StoredGroupedMessageBatch.java | 2 +- .../cradle/messages/StoredMessage.java | 17 +++++--- .../cradle/messages/StoredMessageBatch.java | 6 ++- .../serialization/MessageSerializer.java | 40 +++++++++++-------- .../serialization/MessagesSizeCalculator.java | 16 +++----- .../exactpro/cradle/utils/MessageUtils.java | 28 ++++++++++++- .../GroupedMessageBatchToStoreJoinTest.java | 15 ++++--- .../GroupedMessageBatchToStoreTest.java | 2 +- .../messages/MessageBatchToStoreJoinTest.java | 28 ++++++------- .../messages/MessageBatchToStoreTest.java | 2 +- .../cradle/utils/MessageUtilsTest.java | 4 +- .../utils/SerializationMessageTest.java | 10 +++-- 18 files changed, 141 insertions(+), 87 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java index 73dc22b4e..c3f2f4316 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java @@ -47,7 +47,7 @@ public static SerializedEntity toSerializedEntity (Gr logger.debug("Creating entity from grouped message batch '{}'", group); - SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch.getMessages()); + SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); byte[] batchContent = serializedEntityData.getSerializedData(); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java index caf49026f..7ec774cd1 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java @@ -20,7 +20,10 @@ import com.exactpro.cradle.PageId; import com.exactpro.cradle.cassandra.dao.SerializedEntity; import com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.MessageBatchToStore; +import com.exactpro.cradle.messages.StoredMessage; +import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.serialization.SerializedEntityData; import com.exactpro.cradle.utils.CompressionUtils; import com.exactpro.cradle.utils.MessageUtils; @@ -44,7 +47,7 @@ public static SerializedEntity toSerializedEntity(MessageBat logger.debug("Creating entity from message batch '{}'", batch.getId()); MessageBatchEntity.MessageBatchEntityBuilder builder = MessageBatchEntity.builder(); - SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch.getMessages()); + SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); byte[] batchContent = serializedEntityData.getSerializedData(); builder.setUncompressedContentSize(batchContent.length); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/CradleMessage.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/CradleMessage.java index 6ed954c26..82e395104 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/CradleMessage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/CradleMessage.java @@ -18,6 +18,7 @@ import com.exactpro.cradle.BookId; import com.exactpro.cradle.Direction; +import com.exactpro.cradle.serialization.MessagesSizeCalculator; import java.time.Instant; @@ -82,4 +83,11 @@ public default long getSequence() { StoredMessageId id = getId(); return id != null ? id.getSequence() : null; } + + /** + * @return serialized size of this message in bytes when it'll be added to batch + */ + default int getSerializedSize() { + return MessagesSizeCalculator.calculateMessageSizeInBatch(this); + } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java index 920a780ce..ebd7167de 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java @@ -24,7 +24,6 @@ import org.slf4j.LoggerFactory; import java.time.Instant; -import java.time.LocalDate; import java.util.Collection; import java.util.HashMap; import java.util.Map; @@ -67,7 +66,7 @@ public int getSpaceLeft() { * @return true if batch has enough space to hold given message */ public boolean hasSpace(MessageToStore message) { - return hasSpace(MessagesSizeCalculator.calculateMessageSize(message)); + return hasSpace(message.getSerializedSize()); } private boolean hasSpace(int messageSize) { @@ -81,9 +80,8 @@ private boolean hasSpace(int messageSize) { * @return immutable message object with assigned ID * @throws CradleStorageException if message cannot be added to the batch due to verification failure */ - //FIXME: java.util.TreeSet.add() 68,759 ms (28.3%) jmh test public StoredMessage addMessage(MessageToStore message) throws CradleStorageException { - int expMsgSize = MessagesSizeCalculator.calculateMessageSizeInBatch(message); + int expMsgSize = message.getSerializedSize(); if (!hasSpace(expMsgSize)) throw new CradleStorageException("Batch has not enough space to hold given message"); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageBatchToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageBatchToStore.java index aaab0c712..94796f3da 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageBatchToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageBatchToStore.java @@ -16,17 +16,14 @@ package com.exactpro.cradle.messages; -import java.time.Instant; -import java.time.LocalDate; -import java.time.temporal.ChronoUnit; -import java.util.*; - import com.exactpro.cradle.serialization.MessagesSizeCalculator; +import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.TimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import com.exactpro.cradle.utils.CradleStorageException; +import java.time.Instant; +import java.util.Objects; /** * Holds information about batch of messages to be stored in Cradle. @@ -78,7 +75,7 @@ public int getSpaceLeft() */ public boolean hasSpace(MessageToStore message) { - return hasSpace(MessagesSizeCalculator.calculateMessageSize(message)); + return hasSpace(message.getSerializedSize()); } private boolean hasSpace(int messageSize) @@ -96,7 +93,7 @@ private boolean hasSpace(int messageSize) */ public StoredMessage addMessage(MessageToStore message) throws CradleStorageException { - int expMsgSize = MessagesSizeCalculator.calculateMessageSizeInBatch(message); + int expMsgSize = message.getSerializedSize(); if (!hasSpace(expMsgSize)) throw new CradleStorageException("Batch has not enough space to hold given message"); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java index c57a58fad..b9ffee645 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java @@ -16,12 +16,13 @@ package com.exactpro.cradle.messages; -import java.util.Arrays; - +import com.exactpro.cradle.serialization.MessagesSizeCalculator; import com.exactpro.cradle.utils.CompressionUtils; import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.MessageUtils; +import java.util.Arrays; + /** * Object to hold information about one message prepared to be stored in Cradle */ @@ -30,17 +31,23 @@ public class MessageToStore implements CradleMessage { private final String protocol; private final byte[] content; private MessageMetadata metadata; + private int serializedSize; MessageToStore(StoredMessageId id, String protocol, byte[] content) throws CradleStorageException { + this(id, protocol, content, null); + } + + MessageToStore(StoredMessageId id, String protocol, byte[] content, MessageMetadata metadata) throws CradleStorageException { this.id = id; this.protocol = protocol; this.content = content; + this.metadata = metadata; MessageUtils.validateMessage(this); + this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); } public MessageToStore(MessageToStore copyFrom) throws CradleStorageException { - this(copyFrom.getId(), copyFrom.getProtocol(), copyFrom.getContent()); - this.metadata = copyFrom.getMetadata() != null ? new MessageMetadata(copyFrom.getMetadata()) : null; + this(copyFrom.getId(), copyFrom.getProtocol(), copyFrom.getContent(), copyFrom.getMetadata() != null ? new MessageMetadata(copyFrom.getMetadata()) : null); } public static MessageToStoreBuilder builder() { @@ -68,14 +75,21 @@ public MessageMetadata getMetadata() { return metadata; } + @Override + public int getSerializedSize() { + return serializedSize; + } + public void setMetadata(MessageMetadata metadata) { this.metadata = metadata; + this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); } public void addMetadata(String key, String value) { if (metadata == null) metadata = new MessageMetadata(); metadata.add(key, value); + this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java index 96dca1532..0bf0d0e66 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java @@ -52,7 +52,7 @@ public StoredGroupedMessageBatch(String group, Collection message batchSize = MESSAGE_BATCH_CONST_VALUE; return; } - batchSize = MessagesSizeCalculator.calculateMessageBatchSize(messages).total; + batchSize = MessagesSizeCalculator.calculateMessageBatchSize(messages); } public String getGroup() { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessage.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessage.java index c6e514b85..33178b95a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessage.java @@ -16,16 +16,17 @@ package com.exactpro.cradle.messages; -import java.io.Serializable; -import java.time.Instant; -import java.util.Arrays; -import java.util.Objects; - import com.exactpro.cradle.BookId; import com.exactpro.cradle.Direction; import com.exactpro.cradle.PageId; +import com.exactpro.cradle.serialization.MessagesSizeCalculator; import com.exactpro.cradle.utils.CompressionUtils; +import java.io.Serializable; +import java.time.Instant; +import java.util.Arrays; +import java.util.Objects; + /** * Holds information about one message stored in Cradle. */ @@ -37,6 +38,7 @@ public class StoredMessage implements Serializable, CradleMessage { private final StoredMessageMetadata metadata; private final PageId pageId; private final String protocol; + private final int serializedSize; public StoredMessage(CradleMessage message, StoredMessageId id, PageId pageId) { this(id, message.getProtocol(), message.getContent(), message.getMetadata() != null @@ -53,6 +55,7 @@ protected StoredMessage(StoredMessageId id, String protocol, byte[] content, Sto this.content = content; this.metadata = metadata; this.pageId = pageId; + this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); } /** @@ -107,6 +110,10 @@ public PageId getPageId() { return pageId; } + @Override + public int getSerializedSize() { + return serializedSize; + } @Override public int hashCode() { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessageBatch.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessageBatch.java index 66853eba4..c50144252 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessageBatch.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredMessageBatch.java @@ -26,6 +26,8 @@ import java.util.Collections; import java.util.List; +import static com.exactpro.cradle.serialization.MessagesSizeCalculator.MESSAGE_BATCH_CONST_VALUE; + public class StoredMessageBatch implements MessageBatch { protected StoredMessageId id; @@ -43,11 +45,11 @@ public StoredMessageBatch(Collection messages, PageId pageId, Ins this.recDate = recDate; this.messages = createMessagesList(messages, pageId); if (messages == null || messages.isEmpty()) { - batchSize = MessagesSizeCalculator.calculateMessageBatchSize(Collections.emptyList()).total; + batchSize = MESSAGE_BATCH_CONST_VALUE; return; } id = this.messages.get(0).getId(); - batchSize = MessagesSizeCalculator.calculateMessageBatchSize(messages).total; + batchSize = MessagesSizeCalculator.calculateMessageBatchSize(messages); } @Override diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java index 369c15160..9c9e7f67a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java @@ -16,6 +16,8 @@ package com.exactpro.cradle.serialization; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.MessageBatchToStore; import com.exactpro.cradle.messages.StoredMessage; import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.messages.StoredMessageMetadata; @@ -26,40 +28,46 @@ import java.util.List; import java.util.Map; -import static com.exactpro.cradle.serialization.Serialization.MessageBatchConst.*; -import static com.exactpro.cradle.serialization.SerializationUtils.*; +import static com.exactpro.cradle.serialization.MessagesSizeCalculator.MESSAGE_LENGTH_IN_BATCH; +import static com.exactpro.cradle.serialization.Serialization.MessageBatchConst.MESSAGE_BATCH_MAGIC; +import static com.exactpro.cradle.serialization.Serialization.MessageBatchConst.MESSAGE_MAGIC; +import static com.exactpro.cradle.serialization.Serialization.MessageBatchConst.MESSAGE_PROTOCOL_VER; +import static com.exactpro.cradle.serialization.SerializationUtils.printBody; +import static com.exactpro.cradle.serialization.SerializationUtils.printInstant; +import static com.exactpro.cradle.serialization.SerializationUtils.printString; public class MessageSerializer { + public SerializedEntityData serializeBatch(MessageBatchToStore batch) throws SerializationException { + return serializeBatch(batch.getMessages(), batch.getBatchSize()); + } + + public SerializedEntityData serializeBatch(GroupedMessageBatchToStore batch) throws SerializationException { + return serializeBatch(batch.getMessages(), batch.getBatchSize()); + } - //FIXME: com.exactpro.cradle.serialization.MessageSerializer.serializeBatch() 31,319 ms (20.6%) public SerializedEntityData serializeBatch(Collection batch) throws SerializationException { - SerializationBatchSizes messageBatchSizes = MessagesSizeCalculator.calculateMessageBatchSize(batch); - ByteBuffer buffer = ByteBuffer.allocate(messageBatchSizes.total); - - List serializedMessageMetadata = this.serializeBatch(batch, buffer, messageBatchSizes); + return serializeBatch(batch, MessagesSizeCalculator.calculateMessageBatchSize(batch)); + } + private SerializedEntityData serializeBatch(Collection batch, int batchSize) throws SerializationException { + ByteBuffer buffer = ByteBuffer.allocate(batchSize); + List serializedMessageMetadata = this.serializeBatch(batch, buffer); return new SerializedEntityData(serializedMessageMetadata, buffer.array()); } //FIXME: com.exactpro.cradle.serialization.MessageSerializer.serializeBatch() 15,597 ms (10.3%) public List serializeBatch( - Collection batch, ByteBuffer buffer, SerializationBatchSizes messageBatchSizes + Collection batch, ByteBuffer buffer ) throws SerializationException { - - if (messageBatchSizes == null) { - messageBatchSizes = MessagesSizeCalculator.calculateMessageBatchSize(batch); - } - List serializedMessageMetadata = new ArrayList<>(batch.size()); buffer.putInt(MESSAGE_BATCH_MAGIC); buffer.put(MESSAGE_PROTOCOL_VER); - + buffer.putInt(batch.size()); int i = 0; for (StoredMessage message : batch) { - int messageSize = messageBatchSizes.entities[i]; - + int messageSize = message.getSerializedSize() - MESSAGE_LENGTH_IN_BATCH; buffer.putInt(messageSize); this.serialize(message, buffer); serializedMessageMetadata.add(new SerializedEntityMetadata(message.getTimestamp(), messageSize)); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java index 92a4610c1..b7ae27bd9 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessagesSizeCalculator.java @@ -74,20 +74,14 @@ public static int calculateStringSize(String str) { return (str != null ? str.getBytes(StandardCharsets.UTF_8).length : 0) + 4; } - //FIXME: com.exactpro.cradle.serialization.MessagesSizeCalculator.calculateMessageBatchSize() 12,657 ms (8.3%) - public static SerializationBatchSizes calculateMessageBatchSize(Collection message) { + public static int calculateMessageBatchSize(Collection message) { + var total = MESSAGE_BATCH_CONST_VALUE; - SerializationBatchSizes sizes = new SerializationBatchSizes(message.size()); - sizes.total = MESSAGE_BATCH_CONST_VALUE; - - int i = 0; for (CradleMessage storedMessage : message) { - sizes.entities[i] = calculateMessageSize(storedMessage); - sizes.total += MESSAGE_LENGTH_IN_BATCH + sizes.entities[i]; - i++; + total += storedMessage.getSerializedSize(); } - return sizes; + return total; } - + } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java b/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java index 5472127d8..6a5497aa5 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java @@ -18,6 +18,8 @@ import com.exactpro.cradle.BookId; import com.exactpro.cradle.messages.CradleMessage; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.MessageBatchToStore; import com.exactpro.cradle.messages.StoredMessage; import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.serialization.MessageCommonParams; @@ -60,14 +62,36 @@ public static void validateMessage(CradleMessage message) throws CradleStorageEx throw new CradleStorageException("Message must have content"); } + /** + * Serializes batch messages, skipping non-meaningful or calculable fields + * + * @param batch to serialize + * @return {@link SerializedEntityData} containing serialized messages. + * @throws IOException if serialization failed + */ + public static SerializedEntityData serializeMessages(MessageBatchToStore batch) throws IOException { + return serializer.serializeBatch(batch); + } + + /** + * Serializes batch messages, skipping non-meaningful or calculable fields + * + * @param batch to serialize + * @return {@link SerializedEntityData} containing serialized messages. + * @throws IOException if serialization failed + */ + public static SerializedEntityData serializeMessages(GroupedMessageBatchToStore batch) throws IOException { + return serializer.serializeBatch(batch); + } + /** * Serializes messages, skipping non-meaningful or calculable fields + * * @param messages to serialize * @return {@link SerializedEntityData} containing serialized messages. * @throws IOException if serialization failed */ - public static SerializedEntityData serializeMessages(Collection messages) throws IOException - { + public static SerializedEntityData serializeMessages(Collection messages) throws IOException { return serializer.serializeBatch(messages); } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java index 100e149d6..0e8614768 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java @@ -26,7 +26,9 @@ import java.time.Instant; -import static org.testng.Assert.*; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class GroupedMessageBatchToStoreJoinTest { @@ -42,7 +44,7 @@ public void testJoinEmptyBatchWithOther() throws CradleStorageException, Seriali assertTrue(emptyBatch.addBatch(batch)); assertEquals(emptyBatch.getMessageCount(), 5); - assertEquals(emptyBatch.getBatchSize(), getBatchSize(emptyBatch)); + assertEquals(emptyBatch.getBatchSize(), emptyBatch.getBatchSize()); assertEquals(emptyBatch.getGroup(), groupName); } @@ -94,10 +96,10 @@ public void testAddBatchLessThanLimit() throws CradleStorageException, Serializa GroupedMessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, 5, 5, groupName, protocol); GroupedMessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH.plusMillis(5), 5, 5, groupName, protocol); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertTrue(first.addBatch(second)); assertEquals(first.getMessageCount(), 10); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertEquals(first.getGroup(), groupName); } @@ -110,7 +112,7 @@ public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, Ser assertFalse(first.addBatch(second)); assertEquals(first.getMessageCount(), 1); assertEquals(first.getBatchSize(), sizeBefore); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertEquals(first.getGroup(), groupName); } @@ -213,7 +215,4 @@ static GroupedMessageBatchToStore createFullBySizeBatch(BookId bookId, protocol); } - private int getBatchSize(StoredGroupedMessageBatch batch) throws SerializationException { - return MessagesSizeCalculator.calculateMessageBatchSize(batch.getMessages()).total; - } } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java index 3ca65dda9..4859202a5 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreTest.java @@ -337,7 +337,7 @@ public void batchSerialization() throws CradleStorageException, IOException .build()); StoredMessage storedMsg = batch.getFirstMessage(); - byte[] bytes = MessageUtils.serializeMessages(batch.getMessages()).getSerializedData(); + byte[] bytes = MessageUtils.serializeMessages(batch).getSerializedData(); StoredMessage msg = MessageUtils.deserializeMessages(bytes, book).iterator().next(); Assert.assertEquals(msg, storedMsg, "Message should be completely serialized/deserialized"); } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java index 18337b36d..2b339511c 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java @@ -16,20 +16,19 @@ package com.exactpro.cradle.messages; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; - -import java.time.Instant; - +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.Direction; import com.exactpro.cradle.serialization.MessagesSizeCalculator; import com.exactpro.cradle.serialization.SerializationException; +import com.exactpro.cradle.utils.CradleStorageException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; -import com.exactpro.cradle.Direction; -import com.exactpro.cradle.BookId; -import com.exactpro.cradle.utils.CradleStorageException; +import java.time.Instant; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; public class MessageBatchToStoreJoinTest { @@ -44,7 +43,7 @@ public void testJoinEmptyBatchWithOther() throws CradleStorageException, Seriali assertTrue(emptyBatch.addBatch(batch)); assertEquals(emptyBatch.getMessageCount(), 5); - assertEquals(emptyBatch.getBatchSize(), getBatchSize(emptyBatch)); + assertEquals(emptyBatch.getBatchSize(), emptyBatch.getBatchSize()); assertEquals(emptyBatch.getSessionAlias(), "test"); assertEquals(emptyBatch.getDirection(), Direction.FIRST); } @@ -99,10 +98,10 @@ public void testAddBatchLessThanLimit() throws CradleStorageException, Serializa MessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol, 5, 5); MessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH.plusMillis(5), protocol, 5, 5); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertTrue(first.addBatch(second)); assertEquals(first.getMessageCount(), 10); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertEquals(first.getSessionAlias(), "test"); assertEquals(first.getDirection(), Direction.FIRST); } @@ -116,7 +115,7 @@ public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, Ser assertFalse(first.addBatch(second)); assertEquals(first.getMessageCount(), 1); assertEquals(first.getBatchSize(), sizeBefore); - assertEquals(first.getBatchSize(), getBatchSize(first)); + assertEquals(first.getBatchSize(), first.getBatchSize()); assertEquals(first.getSessionAlias(), "test"); assertEquals(first.getDirection(), Direction.FIRST); } @@ -229,7 +228,4 @@ static MessageBatchToStore createFullBySizeBatch(BookId bookId, MessagesSizeCalculator.calculateStringSize(direction.getLabel()))); } - private int getBatchSize(StoredMessageBatch batch) throws SerializationException { - return MessagesSizeCalculator.calculateMessageBatchSize(batch.getMessages()).total; - } } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreTest.java index 9209c66db..2a988cb2a 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreTest.java @@ -368,7 +368,7 @@ public void batchSerialization() throws CradleStorageException, IOException .content(messageContent) .build(), MAX_SIZE); StoredMessage storedMsg = batch.getFirstMessage(); - byte[] bytes = MessageUtils.serializeMessages(batch.getMessages()).getSerializedData(); + byte[] bytes = MessageUtils.serializeMessages(batch).getSerializedData(); StoredMessage msg = MessageUtils.deserializeMessages(bytes, batch.id).iterator().next(); Assert.assertEquals(msg, storedMsg, "Message should be completely serialized/deserialized"); } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/utils/MessageUtilsTest.java b/cradle-core/src/test/java/com/exactpro/cradle/utils/MessageUtilsTest.java index 478cce929..c192d7bca 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/utils/MessageUtilsTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/utils/MessageUtilsTest.java @@ -70,7 +70,7 @@ public void prepare() throws CradleStorageException @Test public void messageIds() throws IOException { - byte[] bytes = MessageUtils.serializeMessages(batch.getMessages()).getSerializedData(); + byte[] bytes = MessageUtils.serializeMessages(batch).getSerializedData(); Collection restored = MessageUtils.deserializeMessages(bytes, batch.getId()); Iterator it = restored.iterator(); @@ -82,7 +82,7 @@ public void messageIds() throws IOException @Test public void oneMessageId() throws IOException { - byte[] bytes = MessageUtils.serializeMessages(batch.getMessages()).getSerializedData(); + byte[] bytes = MessageUtils.serializeMessages(batch).getSerializedData(); StoredMessage restored = MessageUtils.deserializeOneMessage(bytes, msg2.getId()); Assert.assertEquals(restored.getId(), msg2.getId(), "ID of requested message"); } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/utils/SerializationMessageTest.java b/cradle-core/src/test/java/com/exactpro/cradle/utils/SerializationMessageTest.java index 3393a70ce..0d3442b63 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/utils/SerializationMessageTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/utils/SerializationMessageTest.java @@ -20,7 +20,11 @@ import com.exactpro.cradle.Direction; import com.exactpro.cradle.messages.StoredMessage; import com.exactpro.cradle.messages.StoredMessageBuilder; -import com.exactpro.cradle.serialization.*; +import com.exactpro.cradle.serialization.MessageCommonParams; +import com.exactpro.cradle.serialization.MessageDeserializer; +import com.exactpro.cradle.serialization.MessageSerializer; +import com.exactpro.cradle.serialization.MessagesSizeCalculator; +import com.exactpro.cradle.serialization.SerializationException; import org.testng.Assert; import org.testng.annotations.Test; @@ -226,7 +230,7 @@ public void checkMessageBatchLength() throws SerializationException { ByteBuffer buffer = ByteBuffer.allocate(10_000); List batch = getBatch(); - serializer.serializeBatch(batch, buffer, null); - Assert.assertEquals(buffer.position(), MessagesSizeCalculator.calculateMessageBatchSize(batch).total); + serializer.serializeBatch(batch, buffer); + Assert.assertEquals(buffer.position(), MessagesSizeCalculator.calculateMessageBatchSize(batch)); } } From 26a1aad9154c76780ac2d96c5e05fd4fafb521b5 Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Mon, 20 Mar 2023 16:20:13 +0400 Subject: [PATCH 12/31] Add option for LZ4 compression --- cradle-core/build.gradle | 1 + .../cradle/utils/CompressionUtils.java | 59 +++++++++++++++---- 2 files changed, 47 insertions(+), 13 deletions(-) diff --git a/cradle-core/build.gradle b/cradle-core/build.gradle index 1c97b01e5..501249a5c 100644 --- a/cradle-core/build.gradle +++ b/cradle-core/build.gradle @@ -10,6 +10,7 @@ dependencies { testImplementation 'org.testng:testng:7.1.0' testImplementation 'org.assertj:assertj-core:3.12.2' implementation 'com.google.guava:guava' + implementation 'org.lz4:lz4-java:1.8.0' } test { diff --git a/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionUtils.java b/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionUtils.java index a6ccf1c25..89c99e13e 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionUtils.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionUtils.java @@ -16,21 +16,49 @@ package com.exactpro.cradle.utils; +import net.jpountz.lz4.LZ4Compressor; +import net.jpountz.lz4.LZ4Factory; +import net.jpountz.lz4.LZ4FastDecompressor; + import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.zip.DataFormatException; import java.util.zip.Deflater; import java.util.zip.Inflater; -public class CompressionUtils -{ +public class CompressionUtils { public static String EOL = "\r\n"; public static int BUFFER_SIZE = 4096; - - public static byte[] compressData(byte[] data) throws IOException - { - try(ByteArrayOutputStream outputStream = new ByteArrayOutputStream(data.length)) - { + private static final boolean USE_LZ4 = Boolean.getBoolean("USE_LZ4"); + private static final LZ4Compressor COMPRESSOR = LZ4Factory.fastestInstance().fastCompressor(); + private static final LZ4FastDecompressor DECOMPRESSOR = LZ4Factory.fastestInstance().fastDecompressor(); + + static { + if (USE_LZ4) { + System.out.println("Using LZ4 compression"); + } else { + System.out.println("Using ZIP compression"); + } + } + + public static byte[] compressData(byte[] data) throws IOException { + return USE_LZ4 ? compressLz4(data) : compressZip(data); + } + + public static byte[] compressLz4(byte[] data) { + var length = data.length; + var maxLength = COMPRESSOR.maxCompressedLength(length); + var compressed = new byte[Integer.BYTES + maxLength]; + var newLength = COMPRESSOR.compress(data, 0, length, compressed, Integer.BYTES, maxLength); + var buffer = ByteBuffer.wrap(compressed); + buffer.putInt(0, length); + return Arrays.copyOf(compressed, Integer.BYTES + newLength); + } + + public static byte[] compressZip(byte[] data) throws IOException { + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream(data.length)) { Deflater deflater = new Deflater(); deflater.setInput(data); deflater.finish(); @@ -44,15 +72,20 @@ public static byte[] compressData(byte[] data) throws IOException } } - public static byte[] decompressData(byte[] data) throws IOException, DataFormatException - { - try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream(data.length)) - { + public static byte[] decompressData(byte[] data) throws IOException, DataFormatException { + return USE_LZ4 ? decompressLz4(data) : decompressZip(data); + } + + public static byte[] decompressLz4(byte[] data) { + return DECOMPRESSOR.decompress(data, Integer.BYTES, ByteBuffer.wrap(data).getInt()); + } + + public static byte[] decompressZip(byte[] data) throws IOException, DataFormatException { + try (ByteArrayOutputStream outputStream = new ByteArrayOutputStream(data.length)) { Inflater inflater = new Inflater(); inflater.setInput(data); byte[] buffer = new byte[BUFFER_SIZE]; - while (!inflater.finished()) - { + while (!inflater.finished()) { int count = inflater.inflate(buffer); outputStream.write(buffer, 0, count); } From e7039124a6f2ee8ede7451d170972ce1ae6d6b98 Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Tue, 21 Mar 2023 16:29:31 +0400 Subject: [PATCH 13/31] GroupedMessageBatchToStore > optimize addMessage --- .../messages/GroupedMessageBatchToStore.java | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java index ebd7167de..8e3a7d83b 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java @@ -17,7 +17,6 @@ package com.exactpro.cradle.messages; import com.exactpro.cradle.Direction; -import com.exactpro.cradle.serialization.MessagesSizeCalculator; import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.TimeUtils; import org.slf4j.Logger; @@ -88,11 +87,11 @@ public StoredMessage addMessage(MessageToStore message) throws CradleStorageExce // Checking that the timestamp of a message is not from the future // Other checks have already been done when the MessageToStore was created SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); - Instant now = Instant.now(); - if (message.getTimestamp().isAfter(now)) + long now = System.currentTimeMillis() / 1000; + if (message.getTimestamp().getEpochSecond() > now) throw new CradleStorageException( "Message timestamp (" + TimeUtils.toLocalTimestamp(message.getTimestamp()) + - ") is greater than current timestamp (" + TimeUtils.toLocalTimestamp(now) + ")"); + ") is greater than current timestamp (" + TimeUtils.toLocalTimestamp(Instant.ofEpochSecond(now)) + ")"); long messageSeq; if (bookId == null) { // first message in the batch @@ -143,8 +142,7 @@ public StoredMessage addMessage(MessageToStore message) throws CradleStorageExce StoredMessageId msgId = new StoredMessageId(message.getBookId(), message.getSessionAlias(), message.getDirection(), message.getTimestamp(), messageSeq); StoredMessage msg = new StoredMessage(message, msgId, null); messages.add(msg); - firstMessages.putIfAbsent(sessionKey, msg); - lastMessages.put(sessionKey, msg); + if (lastMessages.put(sessionKey, msg) == null) firstMessages.put(sessionKey, msg); batchSize += expMsgSize; return msg; @@ -183,7 +181,7 @@ public boolean addBatch(GroupedMessageBatchToStore batch) throws CradleStorageEx if (isFull() || batch.isFull()) return false; - int resultSize = batchSize + batch.messages.stream().mapToInt(MessagesSizeCalculator::calculateMessageSizeInBatch).sum(); + int resultSize = batchSize + batch.messages.stream().mapToInt(StoredMessage::getSerializedSize).sum(); if (resultSize > maxBatchSize) { // cannot add because of size limit @@ -193,8 +191,7 @@ public boolean addBatch(GroupedMessageBatchToStore batch) throws CradleStorageEx batch.getMessages().forEach(message -> { messages.add(message); SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); - lastMessages.put(sessionKey, message); - firstMessages.putIfAbsent(sessionKey, message); + if (lastMessages.put(sessionKey, message) == null) firstMessages.put(sessionKey, message); }); this.batchSize = resultSize; return true; @@ -283,4 +280,4 @@ public int hashCode() { return result; } } -} \ No newline at end of file +} From c224fbb1461093b994e9a6e0eed14cd2f2c6bedd Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Fri, 24 Mar 2023 13:35:51 +0400 Subject: [PATCH 14/31] Prepared CassandraStorageSettings for parse json config --- .../cassandra/CassandraCradleStorage.java | 7 +- .../cassandra/CassandraStorageSettings.java | 185 +++++++++++------- .../com/exactpro/cradle/CradleStorage.java | 6 +- 3 files changed, 118 insertions(+), 80 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java index 0c65d8b97..fa28e46ce 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java @@ -111,12 +111,15 @@ public CassandraCradleStorage(CassandraConnectionSettings connectionSettings, Ca this.settings = storageSettings; this.multiRowResultExecPolicy = settings.getMultiRowResultExecutionPolicy(); - if (this.multiRowResultExecPolicy == null) + if (this.multiRowResultExecPolicy == null) { this.multiRowResultExecPolicy = new PageSizeAdjustingPolicy(settings.getResultPageSize(), 2); + } this.singleRowResultExecPolicy = settings.getSingleRowResultExecutionPolicy(); - if (this.singleRowResultExecPolicy == null) + if (this.singleRowResultExecPolicy == null) { this.singleRowResultExecPolicy = new FixedNumberRetryPolicy(5); + } + LOGGER.info("Created cassandra cradle storage via {}", storageSettings); } private static final Consumer NOOP = whatever -> {}; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index 7483f46db..32c0b0950 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -17,70 +17,78 @@ package com.exactpro.cradle.cassandra; import com.datastax.oss.driver.api.core.ConsistencyLevel; -import com.exactpro.cradle.CradleStorage; import com.exactpro.cradle.cassandra.connection.NetworkTopologyStrategy; import com.exactpro.cradle.cassandra.retries.SelectExecutionPolicy; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import static com.exactpro.cradle.CradleStorage.DEFAULT_COMPOSING_SERVICE_THREADS; +import static com.exactpro.cradle.CradleStorage.DEFAULT_MAX_MESSAGE_BATCH_SIZE; +import static com.exactpro.cradle.CradleStorage.DEFAULT_MAX_TEST_EVENT_BATCH_SIZE; +@JsonIgnoreProperties(ignoreUnknown = true) public class CassandraStorageSettings { public static final String SCHEMA_VERSION = "5.3.0"; - public static final long DEFAULT_TIMEOUT = 5000; + public static final ConsistencyLevel DEFAULT_CONSISTENCY_LEVEL = ConsistencyLevel.LOCAL_QUORUM; - public static final int DEFAULT_KEYSPACE_REPL_FACTOR = 1, - DEFAULT_MAX_PARALLEL_QUERIES = 500, - DEFAULT_RESULT_PAGE_SIZE = 0, - DEFAULT_MAX_UNCOMPRESSED_MESSAGE_BATCH_SIZE = 5 * 1024, - DEFAULT_MAX_UNCOMPRESSED_TEST_EVENT_SIZE = 5 * 1024, - DEFAULT_SESSIONS_CACHE_SIZE = 100, - DEFAULT_SCOPES_CACHE_SIZE = 10, - DEFAULT_PAGE_SESSION_CACHE_SIZE = 100, - DEFAULT_PAGE_SCOPES_CACHE_SIZE = 100, - DEFAULT_SESSION_STATISTICS_CACHE_SIZE = 10_000, - DEFAULT_GROUPS_CACHE_SIZE = 10_000, - DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE = 5_000, - DEFAULT_PAGE_GROUPS_CACHE_SIZE = 10_000, - DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS = 1000; - public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000, - DEFAULT_EVENT_BATCH_DURATION_MILLIS = 5_000; + public static final int DEFAULT_KEYSPACE_REPL_FACTOR = 1; + public static final int DEFAULT_MAX_PARALLEL_QUERIES = 500; + public static final int DEFAULT_RESULT_PAGE_SIZE = 0; + public static final int DEFAULT_MAX_UNCOMPRESSED_MESSAGE_BATCH_SIZE = 5 * 1024; + public static final int DEFAULT_MAX_UNCOMPRESSED_TEST_EVENT_SIZE = 5 * 1024; + public static final int DEFAULT_SESSIONS_CACHE_SIZE = 100; + public static final int DEFAULT_SCOPES_CACHE_SIZE = 10; + public static final int DEFAULT_PAGE_SESSION_CACHE_SIZE = 100; + public static final int DEFAULT_PAGE_SCOPES_CACHE_SIZE = 100; + public static final int DEFAULT_SESSION_STATISTICS_CACHE_SIZE = 10_000; + public static final int DEFAULT_GROUPS_CACHE_SIZE = 10_000; + public static final int DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE = 5_000; + public static final int DEFAULT_PAGE_GROUPS_CACHE_SIZE = 10_000; + public static final int DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS = 1000; + public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000; + public static final long DEFAULT_EVENT_BATCH_DURATION_MILLIS = 5_000; + public static final long DEFAULT_TIMEOUT = 5000; public static final boolean DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS = true; - private final NetworkTopologyStrategy networkTopologyStrategy; - private final long timeout; - private final ConsistencyLevel writeConsistencyLevel, - readConsistencyLevel; - private String keyspace, - schemaVersion; - private int keyspaceReplicationFactor; - - private int maxParallelQueries, //FIXME: remove - resultPageSize, - maxMessageBatchSize, - maxUncompressedMessageBatchSize, - maxTestEventBatchSize, - maxUncompressedTestEventSize, - sessionsCacheSize, - scopesCacheSize, - pageSessionsCacheSize, - pageScopesCacheSize, - sessionStatisticsCacheSize, - pageGroupsCacheSize, - groupsCacheSize, - eventBatchDurationCacheSize, - counterPersistenceInterval, - composingServiceThreads; - - private SelectExecutionPolicy multiRowResultExecutionPolicy, singleRowResultExecutionPolicy; - - private long bookRefreshIntervalMillis, eventBatchDurationMillis; - - private boolean storeIndividualMessageSessions; - - public CassandraStorageSettings() { - this(null, DEFAULT_TIMEOUT, DEFAULT_CONSISTENCY_LEVEL, DEFAULT_CONSISTENCY_LEVEL); - } + @JsonIgnore + private NetworkTopologyStrategy networkTopologyStrategy; + private long timeout = DEFAULT_TIMEOUT; + @JsonIgnore + private ConsistencyLevel writeConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; + @JsonIgnore + private ConsistencyLevel readConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; + private String keyspace; + private String schemaVersion = SCHEMA_VERSION; + private int keyspaceReplicationFactor = DEFAULT_KEYSPACE_REPL_FACTOR; + + private int maxParallelQueries = DEFAULT_MAX_PARALLEL_QUERIES; // FIXME: remove + private int resultPageSize = DEFAULT_RESULT_PAGE_SIZE; + private int maxMessageBatchSize = DEFAULT_MAX_MESSAGE_BATCH_SIZE; + private int maxUncompressedMessageBatchSize = DEFAULT_MAX_UNCOMPRESSED_MESSAGE_BATCH_SIZE; + private int maxTestEventBatchSize = DEFAULT_MAX_TEST_EVENT_BATCH_SIZE; + private int maxUncompressedTestEventSize = DEFAULT_MAX_UNCOMPRESSED_TEST_EVENT_SIZE; + private int sessionsCacheSize = DEFAULT_SESSIONS_CACHE_SIZE; + private int scopesCacheSize = DEFAULT_SCOPES_CACHE_SIZE; + private int pageSessionsCacheSize = DEFAULT_PAGE_SESSION_CACHE_SIZE; + private int pageScopesCacheSize = DEFAULT_PAGE_SCOPES_CACHE_SIZE; + private int sessionStatisticsCacheSize = DEFAULT_SESSION_STATISTICS_CACHE_SIZE; + private int pageGroupsCacheSize = DEFAULT_PAGE_GROUPS_CACHE_SIZE; + private int groupsCacheSize = DEFAULT_GROUPS_CACHE_SIZE; + private int eventBatchDurationCacheSize = DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE; + private int counterPersistenceInterval = DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS; + private int composingServiceThreads = DEFAULT_COMPOSING_SERVICE_THREADS; + + private SelectExecutionPolicy multiRowResultExecutionPolicy; + private SelectExecutionPolicy singleRowResultExecutionPolicy; + + private long bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; + private long eventBatchDurationMillis = DEFAULT_EVENT_BATCH_DURATION_MILLIS; + + private boolean storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; + + public CassandraStorageSettings() { } public CassandraStorageSettings(long timeout, ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel) { @@ -89,33 +97,11 @@ public CassandraStorageSettings(long timeout, public CassandraStorageSettings(NetworkTopologyStrategy networkTopologyStrategy, long timeout, ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel) { + this(); this.networkTopologyStrategy = networkTopologyStrategy; this.timeout = timeout; this.writeConsistencyLevel = writeConsistencyLevel; this.readConsistencyLevel = readConsistencyLevel; - - this.schemaVersion = SCHEMA_VERSION; - - this.keyspaceReplicationFactor = DEFAULT_KEYSPACE_REPL_FACTOR; - this.maxParallelQueries = DEFAULT_MAX_PARALLEL_QUERIES; - this.resultPageSize = DEFAULT_RESULT_PAGE_SIZE; - this.maxMessageBatchSize = CradleStorage.DEFAULT_MAX_MESSAGE_BATCH_SIZE; - this.maxUncompressedMessageBatchSize = DEFAULT_MAX_UNCOMPRESSED_MESSAGE_BATCH_SIZE; - this.maxTestEventBatchSize = CradleStorage.DEFAULT_MAX_TEST_EVENT_BATCH_SIZE; - this.maxUncompressedTestEventSize = DEFAULT_MAX_UNCOMPRESSED_TEST_EVENT_SIZE; - this.pageSessionsCacheSize = DEFAULT_PAGE_SESSION_CACHE_SIZE; - this.scopesCacheSize = DEFAULT_SCOPES_CACHE_SIZE; - this.pageScopesCacheSize = DEFAULT_PAGE_SCOPES_CACHE_SIZE; - this.counterPersistenceInterval = DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS; - this.composingServiceThreads = DEFAULT_COMPOSING_SERVICE_THREADS; - this.bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; - this.sessionsCacheSize = DEFAULT_SESSIONS_CACHE_SIZE; - this.sessionStatisticsCacheSize = DEFAULT_SESSION_STATISTICS_CACHE_SIZE; - this.pageGroupsCacheSize = DEFAULT_PAGE_GROUPS_CACHE_SIZE; - this.groupsCacheSize = DEFAULT_GROUPS_CACHE_SIZE; - this.eventBatchDurationCacheSize = DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE; - this.eventBatchDurationMillis = DEFAULT_EVENT_BATCH_DURATION_MILLIS; - this.storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; } public CassandraStorageSettings(CassandraStorageSettings settings) { @@ -158,18 +144,33 @@ public NetworkTopologyStrategy getNetworkTopologyStrategy() { return networkTopologyStrategy; } + public void setNetworkTopologyStrategy(NetworkTopologyStrategy networkTopologyStrategy) { + this.networkTopologyStrategy = networkTopologyStrategy; + } + public long getTimeout() { return timeout; } + public void setTimeout(long timeout) { + this.timeout = timeout; + } + public ConsistencyLevel getWriteConsistencyLevel() { return writeConsistencyLevel; } + public void setWriteConsistencyLevel(ConsistencyLevel writeConsistencyLevel) { + this.writeConsistencyLevel = writeConsistencyLevel; + } + public ConsistencyLevel getReadConsistencyLevel() { return readConsistencyLevel; } + public void setReadConsistencyLevel(ConsistencyLevel readConsistencyLevel) { + this.readConsistencyLevel = readConsistencyLevel; + } public String getKeyspace() { return keyspace; @@ -370,4 +371,38 @@ public boolean isStoreIndividualMessageSessions() { public void setStoreIndividualMessageSessions(boolean storeIndividualMessageSessions) { this.storeIndividualMessageSessions = storeIndividualMessageSessions; } + + @Override + public String toString() { + return "CassandraStorageSettings{" + + "networkTopologyStrategy=" + networkTopologyStrategy + + ", timeout=" + timeout + + ", writeConsistencyLevel=" + writeConsistencyLevel + + ", readConsistencyLevel=" + readConsistencyLevel + + ", keyspace='" + keyspace + '\'' + + ", schemaVersion='" + schemaVersion + '\'' + + ", keyspaceReplicationFactor=" + keyspaceReplicationFactor + + ", maxParallelQueries=" + maxParallelQueries + + ", resultPageSize=" + resultPageSize + + ", maxMessageBatchSize=" + maxMessageBatchSize + + ", maxUncompressedMessageBatchSize=" + maxUncompressedMessageBatchSize + + ", maxTestEventBatchSize=" + maxTestEventBatchSize + + ", maxUncompressedTestEventSize=" + maxUncompressedTestEventSize + + ", sessionsCacheSize=" + sessionsCacheSize + + ", scopesCacheSize=" + scopesCacheSize + + ", pageSessionsCacheSize=" + pageSessionsCacheSize + + ", pageScopesCacheSize=" + pageScopesCacheSize + + ", sessionStatisticsCacheSize=" + sessionStatisticsCacheSize + + ", pageGroupsCacheSize=" + pageGroupsCacheSize + + ", groupsCacheSize=" + groupsCacheSize + + ", eventBatchDurationCacheSize=" + eventBatchDurationCacheSize + + ", counterPersistenceInterval=" + counterPersistenceInterval + + ", composingServiceThreads=" + composingServiceThreads + + ", multiRowResultExecutionPolicy=" + multiRowResultExecutionPolicy + + ", singleRowResultExecutionPolicy=" + singleRowResultExecutionPolicy + + ", bookRefreshIntervalMillis=" + bookRefreshIntervalMillis + + ", eventBatchDurationMillis=" + eventBatchDurationMillis + + ", storeIndividualMessageSessions=" + storeIndividualMessageSessions + + '}'; + } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index 0943d837c..dc5a620d7 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -52,9 +52,9 @@ public abstract class CradleStorage private static final Logger logger = LoggerFactory.getLogger(CradleStorage.class); public static final ZoneOffset TIMEZONE_OFFSET = ZoneOffset.UTC; public static final long EMPTY_MESSAGE_INDEX = -1L; - public static final int DEFAULT_MAX_MESSAGE_BATCH_SIZE = 1024*1024, - DEFAULT_MAX_TEST_EVENT_BATCH_SIZE = DEFAULT_MAX_MESSAGE_BATCH_SIZE, - DEFAULT_COMPOSING_SERVICE_THREADS = 5; + public static final int DEFAULT_MAX_MESSAGE_BATCH_SIZE = 1024*1024; + public static final int DEFAULT_MAX_TEST_EVENT_BATCH_SIZE = DEFAULT_MAX_MESSAGE_BATCH_SIZE; + public static final int DEFAULT_COMPOSING_SERVICE_THREADS = 5; private static final ThreadFactory THREAD_FACTORY = new ThreadFactoryBuilder().setNameFormat("cradle-storage-%d").build(); From d1836e14e41023dd91fedb98a671c7e05442b9ad Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 12 Apr 2023 13:11:11 +0400 Subject: [PATCH 15/31] Refactoring after review --- .../cassandra/CassandraStorageSettings.java | 20 +- .../utils/GroupedMessageEntityUtils.java | 13 +- .../utils/MessageBatchEntityUtils.java | 18 +- .../messages/StoredGroupedMessageBatch.java | 190 +++++++++--------- .../GroupedMessageBatchToStoreJoinTest.java | 23 ++- .../messages/MessageBatchToStoreJoinTest.java | 86 ++++---- 6 files changed, 182 insertions(+), 168 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index 7bf4390b2..904ae7f13 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -46,14 +46,12 @@ public class CassandraStorageSettings { public static final int DEFAULT_EVENT_BATCH_DURATION_CACHE_SIZE = 5_000; public static final int DEFAULT_PAGE_GROUPS_CACHE_SIZE = 10_000; public static final int DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS = 1000; - public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000; + public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000; public static final long DEFAULT_EVENT_BATCH_DURATION_MILLIS = 5_000; public static final long DEFAULT_TIMEOUT = 5000; public static final boolean DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS = true; public static final CompressionType DEFAULT_COMPRESSION_TYPE = CompressionType.ZLIB; - - @JsonIgnore private NetworkTopologyStrategy networkTopologyStrategy; private long timeout = DEFAULT_TIMEOUT; @@ -85,7 +83,7 @@ public class CassandraStorageSettings { private SelectExecutionPolicy multiRowResultExecutionPolicy; private SelectExecutionPolicy singleRowResultExecutionPolicy; - private long bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; + private long bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; private long eventBatchDurationMillis = DEFAULT_EVENT_BATCH_DURATION_MILLIS; private boolean storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; @@ -136,7 +134,7 @@ public CassandraStorageSettings(CassandraStorageSettings settings) { this.sessionStatisticsCacheSize = settings.getSessionStatisticsCacheSize(); this.counterPersistenceInterval = settings.getCounterPersistenceInterval(); this.composingServiceThreads = settings.getComposingServiceThreads(); - this.bookRefreshIntervalMillis = settings.getBookRefreshIntervalMillis(); + this.bookRefreshIntervalMillis = settings.getBookRefreshIntervalMillis(); this.eventBatchDurationMillis = settings.getEventBatchDurationMillis(); this.eventBatchDurationCacheSize = settings.getEventBatchDurationCacheSize(); @@ -345,13 +343,13 @@ public void setSingleRowResultExecutionPolicy( this.singleRowResultExecutionPolicy = singleRowResultExecutionPolicy; } - public long getBookRefreshIntervalMillis() { - return bookRefreshIntervalMillis; - } + public long getBookRefreshIntervalMillis() { + return bookRefreshIntervalMillis; + } - public void setBookRefreshIntervalMillis(long bookRefreshIntervalMillis) { - this.bookRefreshIntervalMillis = bookRefreshIntervalMillis; - } + public void setBookRefreshIntervalMillis(long bookRefreshIntervalMillis) { + this.bookRefreshIntervalMillis = bookRefreshIntervalMillis; + } public long getEventBatchDurationMillis() { return eventBatchDurationMillis; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java index 54f75d6b5..1c810ebcd 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java @@ -39,11 +39,12 @@ public class GroupedMessageEntityUtils { private static final Logger logger = LoggerFactory.getLogger(GroupedMessageEntityUtils.class); - public static SerializedEntity toSerializedEntity (GroupedMessageBatchToStore batch, - PageId pageId, - CompressionType compressionType, - int maxUncompressedSize) - throws IOException, CompressException { + public static SerializedEntity toSerializedEntity( + GroupedMessageBatchToStore batch, + PageId pageId, + CompressionType compressionType, + int maxUncompressedSize + ) throws IOException, CompressException { GroupedMessageBatchEntity.GroupedMessageBatchEntityBuilder builder = GroupedMessageBatchEntity.builder(); @@ -57,7 +58,7 @@ public static SerializedEntity toSerializedEntity (Gr builder.setUncompressedContentSize(batchContent.length); boolean compressed = batchContent.length > maxUncompressedSize; - if (compressed) { + if (compressed) { logger.trace("Compressing content of grouped message batch '{}'", group); batchContent = compressionType.compress(batchContent); } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java index e4a34bf44..d3fc7f9d8 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java @@ -41,10 +41,12 @@ public class MessageBatchEntityUtils { private static final Logger logger = LoggerFactory.getLogger(MessageBatchEntityUtils.class); - public static SerializedEntity toSerializedEntity(MessageBatchToStore batch, - PageId pageId, - CompressionType compressionType, - int maxUncompressedSize) throws IOException, CompressException { + public static SerializedEntity toSerializedEntity( + MessageBatchToStore batch, + PageId pageId, + CompressionType compressionType, + int maxUncompressedSize + ) throws IOException, CompressException { logger.debug("Creating entity from message batch '{}'", batch.getId()); MessageBatchEntity.MessageBatchEntityBuilder builder = MessageBatchEntity.builder(); @@ -96,10 +98,10 @@ public static StoredMessageBatch toStoredMessageBatch(MessageBatchEntity entity, private static StoredMessageId createId(MessageBatchEntity entity, BookId bookId) { return new StoredMessageId(bookId, - entity.getSessionAlias(), - Direction.byLabel(entity.getDirection()), - TimeUtils.toInstant(entity.getFirstMessageDate(), entity.getFirstMessageTime()), - entity.getSequence()); + entity.getSessionAlias(), + Direction.byLabel(entity.getDirection()), + TimeUtils.toInstant(entity.getFirstMessageDate(), entity.getFirstMessageTime()), + entity.getSequence()); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java index 7b8c211b8..d9d48c9cf 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/StoredGroupedMessageBatch.java @@ -33,105 +33,107 @@ import static java.util.stream.Collectors.toCollection; public class StoredGroupedMessageBatch { - protected BookId bookId; - private final String group; - protected int batchSize; + protected BookId bookId; + private final String group; + protected int batchSize; private final List messages; - private StoredMessage firstMessage; - private StoredMessage lastMessage; - private final Instant recDate; - - public BookId getBookId() { - return bookId; - } - - public StoredGroupedMessageBatch(String group) { - this(group, null, null, null); - } - - public StoredGroupedMessageBatch(String group, Collection messages, PageId pageId, Instant recDate) { - this.recDate = recDate; - this.group = group; - this.messages = messages == null || messages.isEmpty() - ? new ArrayList<>() - : messages.stream() - .map(msg -> Objects.equals(msg.getPageId(), pageId) - ? msg - : new StoredMessage(msg, msg.getId(), pageId) - ).collect(toCollection(ArrayList::new)); - if (this.messages.isEmpty()) { + private StoredMessage firstMessage; + private StoredMessage lastMessage; + private final Instant recDate; + + public BookId getBookId() { + return bookId; + } + + public StoredGroupedMessageBatch(String group) { + this(group, null, null, null); + } + + public StoredGroupedMessageBatch(String group, Collection messages, PageId pageId, Instant recDate) { + this.recDate = recDate; + this.group = group; + this.messages = messages == null || messages.isEmpty() + ? new ArrayList<>() + : messages.stream() + .map(msg -> Objects.equals(msg.getPageId(), pageId) + ? msg + : new StoredMessage(msg, msg.getId(), pageId) + ).collect(toCollection(ArrayList::new)); + if (this.messages.isEmpty()) { batchSize = MESSAGE_BATCH_CONST_VALUE; - return; - } - this.messages.forEach(this::updateFirstLast); - batchSize = MessagesSizeCalculator.calculateMessageBatchSize(this.messages); - } - - public String getGroup() { - return group; - } - public int getMessageCount() { - return messages.size(); - } - - public int getBatchSize() { - return batchSize; - } - - public Collection getMessages() { - return Collections.unmodifiableCollection(messages); - } - - public Collection getMessagesReverse() { + return; + } + this.messages.forEach(this::updateFirstLast); + batchSize = MessagesSizeCalculator.calculateMessageBatchSize(this.messages); + } + + public String getGroup() { + return group; + } + + public int getMessageCount() { + return messages.size(); + } + + public int getBatchSize() { + return batchSize; + } + + public Collection getMessages() { + return Collections.unmodifiableCollection(messages); + } + + public Collection getMessagesReverse() { return Collections.unmodifiableCollection(Lists.reverse(messages)); - } + } - public StoredMessage getFirstMessage() { + public StoredMessage getFirstMessage() { return firstMessage; - } + } - public StoredMessage getLastMessage() { + public StoredMessage getLastMessage() { return lastMessage; - } - - public Instant getFirstTimestamp() { - StoredMessage m = getFirstMessage(); - return m != null ? m.getTimestamp() : null; - } - - public Instant getLastTimestamp() { - StoredMessage m = getLastMessage(); - return m != null ? m.getTimestamp() : null; - } - - - public Instant getRecDate() { - return recDate; - } - - public boolean isEmpty() { - return messages.isEmpty(); - } - - protected Stream messagesStream() { - return messages.stream(); - } - protected void addMessage(StoredMessage message) { - messages.add(message); - updateFirstLast(message); - } - - protected void addMessages(Collection messages) { - this.messages.addAll(messages); - messages.forEach(this::updateFirstLast); - } - - private void updateFirstLast(StoredMessage message) { - if (firstMessage == null || message.getTimestamp().isBefore(firstMessage.getTimestamp())) { - firstMessage = message; - } - if (lastMessage == null || message.getTimestamp().isAfter(lastMessage.getTimestamp())) { - lastMessage = message; - } - } + } + + public Instant getFirstTimestamp() { + StoredMessage m = getFirstMessage(); + return m != null ? m.getTimestamp() : null; + } + + public Instant getLastTimestamp() { + StoredMessage m = getLastMessage(); + return m != null ? m.getTimestamp() : null; + } + + + public Instant getRecDate() { + return recDate; + } + + public boolean isEmpty() { + return messages.isEmpty(); + } + + protected Stream messagesStream() { + return messages.stream(); + } + + protected void addMessage(StoredMessage message) { + messages.add(message); + updateFirstLast(message); + } + + protected void addMessages(Collection messages) { + this.messages.addAll(messages); + messages.forEach(this::updateFirstLast); + } + + private void updateFirstLast(StoredMessage message) { + if (firstMessage == null || message.getTimestamp().isBefore(firstMessage.getTimestamp())) { + firstMessage = message; + } + if (lastMessage == null || message.getTimestamp().isAfter(lastMessage.getTimestamp())) { + lastMessage = message; + } + } } \ No newline at end of file diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java index 7eb634779..67ffd4e48 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java @@ -19,7 +19,6 @@ import com.exactpro.cradle.BookId; import com.exactpro.cradle.Direction; import com.exactpro.cradle.serialization.MessagesSizeCalculator; -import com.exactpro.cradle.serialization.SerializationException; import com.exactpro.cradle.utils.CradleStorageException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -30,21 +29,21 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -public class GroupedMessageBatchToStoreJoinTest -{ +public class GroupedMessageBatchToStoreJoinTest { private static final BookId bookId = new BookId("test-book"); private static final String groupName = "test-group"; private static final String protocol = "test-protocol"; static final int MAX_SIZE = 1024; @Test - public void testJoinEmptyBatchWithOther() throws CradleStorageException, SerializationException { + public void testJoinEmptyBatchWithOther() throws CradleStorageException { GroupedMessageBatchToStore emptyBatch = createEmptyBatch(groupName); GroupedMessageBatchToStore batch = createBatch(bookId, "test", 1, Direction.FIRST, Instant.EPOCH, 5, 5, groupName, null); assertTrue(emptyBatch.addBatch(batch)); assertEquals(emptyBatch.getMessageCount(), 5); - assertEquals(emptyBatch.getBatchSize(), emptyBatch.getBatchSize()); + assertEquals(emptyBatch.getBatchSize(), getBatchSize(emptyBatch)); + assertEquals(batch.getBatchSize(), emptyBatch.getBatchSize()); assertEquals(emptyBatch.getGroup(), groupName); } @@ -92,19 +91,19 @@ public void testFullBatches(GroupedMessageBatchToStore first, GroupedMessageBatc @Test - public void testAddBatchLessThanLimit() throws CradleStorageException, SerializationException { + public void testAddBatchLessThanLimit() throws CradleStorageException { GroupedMessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, 5, 5, groupName, protocol); GroupedMessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH.plusMillis(5), 5, 5, groupName, protocol); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertTrue(first.addBatch(second)); assertEquals(first.getMessageCount(), 10); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertEquals(first.getGroup(), groupName); } @Test - public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, SerializationException { + public void testAddBatchMoreThanLimitBySize() throws CradleStorageException { GroupedMessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, 1, MAX_SIZE / 2, groupName, protocol); GroupedMessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH, 1, MAX_SIZE / 2, groupName, protocol); @@ -112,7 +111,7 @@ public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, Ser assertFalse(first.addBatch(second)); assertEquals(first.getMessageCount(), 1); assertEquals(first.getBatchSize(), sizeBefore); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertEquals(first.getGroup(), groupName); } @@ -214,4 +213,8 @@ static GroupedMessageBatchToStore createFullBySizeBatch(BookId bookId, group, protocol); } + + private int getBatchSize(StoredGroupedMessageBatch batch) { + return MessagesSizeCalculator.calculateMessageBatchSize(batch.getMessages()); + } } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java index b3ed27482..23feec712 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/MessageBatchToStoreJoinTest.java @@ -19,7 +19,6 @@ import com.exactpro.cradle.BookId; import com.exactpro.cradle.Direction; import com.exactpro.cradle.serialization.MessagesSizeCalculator; -import com.exactpro.cradle.serialization.SerializationException; import com.exactpro.cradle.utils.CradleStorageException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; @@ -30,24 +29,24 @@ import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -public class MessageBatchToStoreJoinTest -{ +public class MessageBatchToStoreJoinTest { private static final BookId bookId = new BookId("testbook"); private static final String protocol = "test-proto"; static final int MAX_SIZE = 1024; @Test - public void testJoinEmptyBatchWithOther() throws CradleStorageException, SerializationException { + public void testJoinEmptyBatchWithOther() throws CradleStorageException { MessageBatchToStore emptyBatch = createEmptyBatch(); MessageBatchToStore batch = createBatch(bookId, "test", 1, Direction.FIRST, Instant.EPOCH, null, 5, 5); assertTrue(emptyBatch.addBatch(batch)); assertEquals(emptyBatch.getMessageCount(), 5); - assertEquals(emptyBatch.getBatchSize(), emptyBatch.getBatchSize()); + assertEquals(emptyBatch.getBatchSize(), getBatchSize(emptyBatch)); + assertEquals(batch.getBatchSize(), batch.getBatchSize()); assertEquals(emptyBatch.getSessionAlias(), "test"); assertEquals(emptyBatch.getDirection(), Direction.FIRST); } - + @Test(dataProvider = "full batches") public void testJoinEmptyBatchWithFull(MessageBatchToStore other) throws CradleStorageException { MessageBatchToStore emptyBatch = createEmptyBatch(); @@ -58,11 +57,10 @@ public void testJoinEmptyBatchWithFull(MessageBatchToStore other) throws CradleS assertEquals(emptyBatch.getDirection(), Direction.FIRST); } - @DataProvider(name = "full batches") public Object[][] fullBatches() throws CradleStorageException { - return new Object[][] { - { createFullBySizeBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol) } + return new Object[][]{ + {createFullBySizeBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol)} }; } @@ -82,8 +80,8 @@ public void testJoinFullBatchWithEmpty(MessageBatchToStore batch) throws CradleS @DataProvider(name = "full batches matrix") public Object[][] fullBatchesMatrix() throws CradleStorageException { MessageBatchToStore fullBySizeBatch = createFullBySizeBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol); - return new Object[][] { - { fullBySizeBatch, fullBySizeBatch } + return new Object[][]{ + {fullBySizeBatch, fullBySizeBatch} }; } @@ -94,20 +92,20 @@ public void testFullBatches(MessageBatchToStore first, MessageBatchToStore secon @Test - public void testAddBatchLessThanLimit() throws CradleStorageException, SerializationException { + public void testAddBatchLessThanLimit() throws CradleStorageException { MessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol, 5, 5); MessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH.plusMillis(5), protocol, 5, 5); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertTrue(first.addBatch(second)); assertEquals(first.getMessageCount(), 10); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertEquals(first.getSessionAlias(), "test"); assertEquals(first.getDirection(), Direction.FIRST); } @Test - public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, SerializationException { + public void testAddBatchMoreThanLimitBySize() throws CradleStorageException { MessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, protocol, 1, MAX_SIZE / 2); MessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH, protocol, 1, MAX_SIZE / 2); @@ -115,20 +113,19 @@ public void testAddBatchMoreThanLimitBySize() throws CradleStorageException, Ser assertFalse(first.addBatch(second)); assertEquals(first.getMessageCount(), 1); assertEquals(first.getBatchSize(), sizeBefore); - assertEquals(first.getBatchSize(), first.getBatchSize()); + assertEquals(first.getBatchSize(), getBatchSize(first)); assertEquals(first.getSessionAlias(), "test"); assertEquals(first.getDirection(), Direction.FIRST); } - - + @Test( - expectedExceptions = CradleStorageException.class, - expectedExceptionsMessageRegExp = "IDs are not compatible.*" + expectedExceptions = CradleStorageException.class, + expectedExceptionsMessageRegExp = "IDs are not compatible.*" ) public void testThrowExceptionOnDifferentBooks() throws CradleStorageException { MessageBatchToStore first = createBatch(bookId, "testA", 0, Direction.FIRST, Instant.EPOCH, protocol, 5, 5); MessageBatchToStore - second = createBatch(new BookId(bookId.getName()+"2"), "testA", 5, Direction.FIRST, Instant.EPOCH, protocol, 5, 5); + second = createBatch(new BookId(bookId.getName() + "2"), "testA", 5, Direction.FIRST, Instant.EPOCH, protocol, 5, 5); first.addBatch(second); } @@ -154,7 +151,7 @@ public void testThrowExceptionOnDifferentDirections() throws CradleStorageExcept first.addBatch(second); } - + @Test( expectedExceptions = CradleStorageException.class, expectedExceptionsMessageRegExp = "Batches are not ordered.*" @@ -177,14 +174,16 @@ public void testThrowExceptionOnUnorderedSequences() throws CradleStorageExcepti first.addBatch(second); } - private static MessageBatchToStore createBatch(BookId bookId, - String sessionAlias, - long startSequence, - Direction direction, - Instant startTimestamp, - String protocol, - int messageCount, - int contentSizePerMessage) throws CradleStorageException { + private static MessageBatchToStore createBatch( + BookId bookId, + String sessionAlias, + long startSequence, + Direction direction, + Instant startTimestamp, + String protocol, + int messageCount, + int contentSizePerMessage + ) throws CradleStorageException { MessageBatchToStore messageBatchToStore = createEmptyBatch(); long begin = startSequence; Instant timestamp = startTimestamp; @@ -200,7 +199,7 @@ private static MessageBatchToStore createBatch(BookId bookId, toStore = toStore.content(new byte[contentSizePerMessage]); } messageBatchToStore.addMessage(toStore.build()); - + timestamp = timestamp.plusMillis(1); } return messageBatchToStore; @@ -210,9 +209,14 @@ private static MessageBatchToStore createEmptyBatch() { return new MessageBatchToStore(MAX_SIZE); } - static MessageBatchToStore createFullBySizeBatch(BookId bookId, - String sessionAlias, long startSequence, Direction direction, Instant startTimestamp, String protocol) throws CradleStorageException { - + static MessageBatchToStore createFullBySizeBatch( + BookId bookId, + String sessionAlias, + long startSequence, + Direction direction, + Instant startTimestamp, + String protocol + ) throws CradleStorageException { return createBatch(bookId, sessionAlias, startSequence, @@ -221,10 +225,14 @@ static MessageBatchToStore createFullBySizeBatch(BookId bookId, protocol, 1, MAX_SIZE - (MessagesSizeCalculator.MESSAGE_BATCH_CONST_VALUE + - MessagesSizeCalculator.MESSAGE_SIZE_CONST_VALUE + - MessagesSizeCalculator.MESSAGE_LENGTH_IN_BATCH + - MessagesSizeCalculator.calculateStringSize(sessionAlias) + - MessagesSizeCalculator.calculateStringSize(protocol) + - MessagesSizeCalculator.calculateStringSize(direction.getLabel()))); + MessagesSizeCalculator.MESSAGE_SIZE_CONST_VALUE + + MessagesSizeCalculator.MESSAGE_LENGTH_IN_BATCH + + MessagesSizeCalculator.calculateStringSize(sessionAlias) + + MessagesSizeCalculator.calculateStringSize(protocol) + + MessagesSizeCalculator.calculateStringSize(direction.getLabel()))); + } + + private int getBatchSize(StoredMessageBatch batch) { + return MessagesSizeCalculator.calculateMessageBatchSize(batch.getMessages()); } } From 1c358296319d18a5cb3c562362c713df10946c11 Mon Sep 17 00:00:00 2001 From: "ivan.druzhinin" Date: Wed, 12 Apr 2023 14:02:53 +0400 Subject: [PATCH 16/31] MessageToStoreBuilder > optimize build() --- .../cradle/messages/MessageToStoreBuilder.java | 12 +++++------- 1 file changed, 5 insertions(+), 7 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStoreBuilder.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStoreBuilder.java index d97166c1f..0b113e5b4 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStoreBuilder.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStoreBuilder.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -16,12 +16,12 @@ package com.exactpro.cradle.messages; -import java.time.Instant; - import com.exactpro.cradle.BookId; import com.exactpro.cradle.Direction; import com.exactpro.cradle.utils.CradleStorageException; +import java.time.Instant; + /** * Builder for MessageToStore object. After calling {@link #build()} method, the builder can be reused to build new message */ @@ -99,9 +99,7 @@ public MessageToStoreBuilder metadata(String key, String value) { public MessageToStore build() throws CradleStorageException { try { - MessageToStore result = createMessageToStore(); - result.setMetadata(metadata); - return result; + return createMessageToStore(); } finally { reset(); } @@ -110,7 +108,7 @@ public MessageToStore build() throws CradleStorageException { protected MessageToStore createMessageToStore() throws CradleStorageException { StoredMessageId id = new StoredMessageId(bookId, sessionAlias, direction, timestamp, sequence); - return new MessageToStore(id, protocol, content); + return new MessageToStore(id, protocol, content, metadata); } protected void reset() { From 61c4c8d6f71b8f5feebf794d998d7e29635e8753 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 12 Apr 2023 16:34:53 +0400 Subject: [PATCH 17/31] Refactoring after review * Added custom magic bytes for LZ4 * Verified empty batch size before add another batch --- .../messages/GroupedMessageBatchToStore.java | 520 +++++++++--------- .../cradle/messages/MessageToStore.java | 49 +- .../cradle/utils/CompressionType.java | 70 ++- .../GroupedMessageBatchToStoreJoinTest.java | 11 + .../cradle/utils/CompressionTypeTest.java | 5 + 5 files changed, 371 insertions(+), 284 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java index ef1c5169f..0b21367dd 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageBatchToStore.java @@ -29,254 +29,274 @@ import java.util.Objects; public class GroupedMessageBatchToStore extends StoredGroupedMessageBatch { - private static final Logger logger = LoggerFactory.getLogger(GroupedMessageBatchToStore.class); - - private final int maxBatchSize; - private final Map firstMessages; - private final Map lastMessages; - - public GroupedMessageBatchToStore(String group, int maxBatchSize) { - super(group); - this.maxBatchSize = maxBatchSize; - this.firstMessages = new HashMap<>(); - this.lastMessages = new HashMap<>(); - } - - /** - * Indicates if the batch cannot hold more messages - * @return true if batch capacity is reached and the batch must be flushed to Cradle - */ - public boolean isFull() { - return batchSize >= maxBatchSize; - } - - /** - * Shows how many bytes the batch can hold till its capacity is reached - * @return number of bytes the batch can hold - */ - public int getSpaceLeft() { - int result = maxBatchSize - batchSize; - return Math.max(result, 0); - } - - /** - * Shows if batch has enough space to hold given message - * @param message to check against batch capacity - * @return true if batch has enough space to hold given message - */ - public boolean hasSpace(MessageToStore message) { - return hasSpace(message.getSerializedSize()); - } - - private boolean hasSpace(int messageSize) { - return batchSize + messageSize <= maxBatchSize; - } - - /** - * Adds message to the batch. Batch will add correct message ID by itself, verifying message to match batch conditions. - * Result of this method should be used for all further operations on the message - * @param message to add to the batch - * @return immutable message object with assigned ID - * @throws CradleStorageException if message cannot be added to the batch due to verification failure - */ - public StoredMessage addMessage(MessageToStore message) throws CradleStorageException { - int expMsgSize = message.getSerializedSize(); - if (!hasSpace(expMsgSize)) - throw new CradleStorageException("Batch has not enough space to hold given message"); - - // Checking that the timestamp of a message is not from the future - // Other checks have already been done when the MessageToStore was created - SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); - long now = System.currentTimeMillis() / 1000; - if (message.getTimestamp().getEpochSecond() > now) - throw new CradleStorageException( - "Message timestamp (" + TimeUtils.toLocalTimestamp(message.getTimestamp()) + - ") is greater than current timestamp (" + TimeUtils.toLocalTimestamp(Instant.ofEpochSecond(now)) + ")"); - - long messageSeq; - if (bookId == null) { // first message in the batch - bookId = message.getBookId(); - if (bookId == null) - throw new CradleStorageException("BookId for the message not set (" + message.getId() + ")"); - messageSeq = message.getSequence(); - verifySequence(messageSeq); - - } else { - if (!bookId.equals(message.getBookId())) - throw new CradleStorageException("Batch contains messages of book '" + bookId + "', " - + "but in your message it is '"+message.getBookId()+"'"); - - StoredMessage lastMessage = lastMessages.get(sessionKey); - if (lastMessage != null) { - if (lastMessage.getTimestamp().isAfter(message.getTimestamp())) { - throw new CradleStorageException(String.format( - "Message timestamp should not be before %s, but in your message it is %s", - lastMessage.getTimestamp(), - message.getTimestamp() - )); - } - - if (message.getSequence() > 0) { // i.e. message sequence is set - messageSeq = message.getSequence(); - if (messageSeq <= lastMessage.getSequence()) { - throw new CradleStorageException(String.format( - "Sequence number should be greater than %d for the batch to contain sequenced messages, but in your message it is %d", - lastMessage.getSequence(), - messageSeq - )); - } - if (messageSeq != lastMessage.getSequence() + 1) { - logger.warn(String.format( - "Expected sequence number %d for the batch to contain strictly sequenced messages, but in your message it is %d", - lastMessage.getSequence() + 1, - messageSeq)); - } - } else - messageSeq = lastMessage.getSequence() + 1; - } else { - messageSeq = message.getSequence(); - verifySequence(messageSeq); - } - } - - StoredMessageId msgId = new StoredMessageId(message.getBookId(), message.getSessionAlias(), message.getDirection(), message.getTimestamp(), messageSeq); - StoredMessage msg = new StoredMessage(message, msgId, null); - addMessage(msg); - if (lastMessages.put(sessionKey, msg) == null) firstMessages.put(sessionKey, msg); - batchSize += expMsgSize; - - return msg; - } - - private static void verifySequence(long messageSeq) throws CradleStorageException { - if (messageSeq < 0) - throw new CradleStorageException("Sequence number for first message in batch cannot be negative"); - } - - /** - * - * @param batch the batch to add to the current one. - * The batch to add must contain message with same group name as the current one. - * The index of the first message in the [batch] should be greater - * than the last message index in the current batch. - * @return true if the result batch meets the restriction for message count and batch size - * @throws CradleStorageException if the batch doesn't meet the requirements regarding inner content - */ - public boolean addBatch(GroupedMessageBatchToStore batch) throws CradleStorageException { - - if (!this.getGroup().equals(batch.getGroup())) - throw new CradleStorageException(String.format("Batch groups differ. Current Group is %s, other Group is %s", getGroup(), batch.getGroup())); - if (batch.isEmpty()) - return true; - - if (isEmpty()) { - this.bookId = batch.getBookId(); - this.batchSize = batch.getBatchSize(); - addMessages(batch.getMessages()); - firstMessages.putAll(batch.firstMessages); - lastMessages.putAll(batch.lastMessages); - return true; - } - - if (isFull() || batch.isFull()) - return false; - - int resultSize = batchSize + batch.messagesStream().mapToInt(StoredMessage::getSerializedSize).sum(); - - if (resultSize > maxBatchSize) { - // cannot add because of size limit - return false; - } - verifyBatch(batch); - batch.getMessages().forEach(message -> { - addMessage(message); - SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); - if (lastMessages.put(sessionKey, message) == null) firstMessages.put(sessionKey, message); - }); - this.batchSize = resultSize; - return true; - } - - public Collection getSessionMessageBatches() throws CradleStorageException{ - Map batches = new HashMap<>(); - for (StoredMessage message: getMessages()) { - SessionKey key = new SessionKey(message.getSessionAlias(), message.getDirection()); - MessageBatchToStore batch = batches.computeIfAbsent(key, k -> new MessageBatchToStore(maxBatchSize)); - - StoredMessageId msgId = new StoredMessageId(message.getBookId(), - message.getSessionAlias(), - message.getDirection(), - message.getTimestamp(), - message.getSequence()); - - MessageToStoreBuilder builder = MessageToStore.builder() - .id(msgId) - .protocol(message.getProtocol()) - .content(message.getContent()); - - if (message.getMetadata() != null) - message.getMetadata().toMap().forEach(builder::metadata); - batch.addMessage(builder.build()); - } - - return batches.values(); - } - - - private void verifyBatch(GroupedMessageBatchToStore otherBatch) throws CradleStorageException { - if (!Objects.equals(bookId, otherBatch.getBookId())) - throw new CradleStorageException(String.format("Batch BookId-s differ. Current BookId is %s, other BookId is %s", bookId, otherBatch.getBookId())); - - if (this.getFirstTimestamp().isAfter(otherBatch.getFirstTimestamp()) || - this.getLastTimestamp().isAfter(otherBatch.getFirstTimestamp())) - throw new CradleStorageException( - String.format("Batches intersect by time. Current batch %s - %s, other batch %s - %s", - this.getFirstTimestamp(), - this.getLastTimestamp(), - otherBatch.getFirstTimestamp(), - otherBatch.getLastTimestamp())); - - for (SessionKey sessionKey : lastMessages.keySet()) { - StoredMessage otherFirstMessage = otherBatch.firstMessages.get(sessionKey); - if (otherFirstMessage == null) - continue; - StoredMessage thisLastMessage = this.lastMessages.get(sessionKey); - - if (thisLastMessage.getTimestamp().isAfter(otherFirstMessage.getTimestamp())) - throw new CradleStorageException(String.format("Batches are not ordered. Current last timestamp: %s; Other first timestamp: %s", - thisLastMessage.getTimestamp(), otherFirstMessage.getTimestamp())); - - if (thisLastMessage.getSequence() >= otherFirstMessage.getSequence()) { - throw new CradleStorageException(String.format("Batches are not ordered. Current last sequence number: %d; Other first sequence number: %d", - thisLastMessage.getSequence(), otherFirstMessage.getSequence())); - } - } - } - - private static class SessionKey { - final String sessionAlias; - final Direction direction; - SessionKey(String sessionAlias, Direction direction) { - this.sessionAlias = sessionAlias; - this.direction = direction; - } - - @Override - public boolean equals(Object o) { - if (! (o instanceof SessionKey)) - return false; - - SessionKey that = (SessionKey) o; - if (!sessionAlias.equals(that.sessionAlias)) - return false; - return direction == that.direction; - } - - @Override - public int hashCode() { - int result = sessionAlias.hashCode(); - result = 31 * result + direction.hashCode(); - return result; - } - } + private static final Logger logger = LoggerFactory.getLogger(GroupedMessageBatchToStore.class); + + private final int maxBatchSize; + private final Map firstMessages; + private final Map lastMessages; + + public GroupedMessageBatchToStore(String group, int maxBatchSize) { + super(group); + this.maxBatchSize = maxBatchSize; + this.firstMessages = new HashMap<>(); + this.lastMessages = new HashMap<>(); + } + + /** + * Indicates if the batch cannot hold more messages + * + * @return true if batch capacity is reached and the batch must be flushed to Cradle + */ + public boolean isFull() { + return batchSize >= maxBatchSize; + } + + /** + * Shows how many bytes the batch can hold till its capacity is reached + * + * @return number of bytes the batch can hold + */ + public int getSpaceLeft() { + int result = maxBatchSize - batchSize; + return Math.max(result, 0); + } + + /** + * Shows if batch has enough space to hold given message + * + * @param message to check against batch capacity + * @return true if batch has enough space to hold given message + */ + public boolean hasSpace(MessageToStore message) { + return hasSpace(message.getSerializedSize()); + } + + private boolean hasSpace(int messageSize) { + return batchSize + messageSize <= maxBatchSize; + } + + /** + * Adds message to the batch. Batch will add correct message ID by itself, verifying message to match batch conditions. + * Result of this method should be used for all further operations on the message + * + * @param message to add to the batch + * @return immutable message object with assigned ID + * @throws CradleStorageException if message cannot be added to the batch due to verification failure + */ + public StoredMessage addMessage(MessageToStore message) throws CradleStorageException { + int expMsgSize = message.getSerializedSize(); + if (!hasSpace(expMsgSize)) + throw new CradleStorageException("Batch has not enough space to hold given message"); + + // Checking that the timestamp of a message is not from the future + // Other checks have already been done when the MessageToStore was created + SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); + long now = System.currentTimeMillis() / 1000; + if (message.getTimestamp().getEpochSecond() > now) + throw new CradleStorageException( + "Message timestamp (" + TimeUtils.toLocalTimestamp(message.getTimestamp()) + + ") is greater than current timestamp (" + TimeUtils.toLocalTimestamp(Instant.ofEpochSecond(now)) + ")"); + + long messageSeq; + if (bookId == null) { // first message in the batch + bookId = message.getBookId(); + if (bookId == null) + throw new CradleStorageException("BookId for the message not set (" + message.getId() + ")"); + messageSeq = message.getSequence(); + verifySequence(messageSeq); + } else { + if (!bookId.equals(message.getBookId())) + throw new CradleStorageException("Batch contains messages of book '" + bookId + "', " + + "but in your message it is '" + message.getBookId() + "'"); + + StoredMessage lastMessage = lastMessages.get(sessionKey); + if (lastMessage != null) { + if (lastMessage.getTimestamp().isAfter(message.getTimestamp())) { + throw new CradleStorageException(String.format( + "Message timestamp should not be before %s, but in your message it is %s", + lastMessage.getTimestamp(), + message.getTimestamp() + )); + } + + if (message.getSequence() > 0) { // i.e. message sequence is set + messageSeq = message.getSequence(); + if (messageSeq <= lastMessage.getSequence()) { + throw new CradleStorageException(String.format( + "Sequence number should be greater than %d for the batch to contain sequenced messages, but in your message it is %d", + lastMessage.getSequence(), + messageSeq + )); + } + if (messageSeq != lastMessage.getSequence() + 1) { + logger.warn(String.format( + "Expected sequence number %d for the batch to contain strictly sequenced messages, but in your message it is %d", + lastMessage.getSequence() + 1, + messageSeq + )); + } + } else + messageSeq = lastMessage.getSequence() + 1; + } else { + messageSeq = message.getSequence(); + verifySequence(messageSeq); + } + } + + StoredMessageId msgId = new StoredMessageId(message.getBookId(), message.getSessionAlias(), message.getDirection(), message.getTimestamp(), messageSeq); + StoredMessage msg = new StoredMessage(message, msgId, null); + addMessage(msg); + if (lastMessages.put(sessionKey, msg) == null) firstMessages.put(sessionKey, msg); + batchSize += expMsgSize; + + return msg; + } + + private static void verifySequence(long messageSeq) throws CradleStorageException { + if (messageSeq < 0) + throw new CradleStorageException("Sequence number for first message in batch cannot be negative"); + } + + /** + * @param batch the batch to add to the current one. + * The batch to add must contain message with same group name as the current one. + * The index of the first message in the [batch] should be greater + * than the last message index in the current batch. + * @return true if the result batch meets the restriction for message count and batch size + * @throws CradleStorageException if the batch doesn't meet the requirements regarding inner content + */ + public boolean addBatch(GroupedMessageBatchToStore batch) throws CradleStorageException { + + if (!this.getGroup().equals(batch.getGroup())) + throw new CradleStorageException(String.format("Batch groups differ. Current Group is %s, other Group is %s", getGroup(), batch.getGroup())); + if (batch.isEmpty()) + return true; + + if (isEmpty()) { + int resultSize = batchSize + calculateMessagesSize(batch); + if (resultSize > maxBatchSize) { + // cannot add because of size limit + return false; + } + + this.bookId = batch.getBookId(); + this.batchSize = batch.getBatchSize(); + addMessages(batch.getMessages()); + firstMessages.putAll(batch.firstMessages); + lastMessages.putAll(batch.lastMessages); + return true; + } + + if (isFull() || batch.isFull()) + return false; + + int resultSize = batchSize + calculateMessagesSize(batch); + if (resultSize > maxBatchSize) { + // cannot add because of size limit + return false; + } + + verifyBatch(batch); + batch.getMessages().forEach(message -> { + addMessage(message); + SessionKey sessionKey = new SessionKey(message.getSessionAlias(), message.getDirection()); + if (lastMessages.put(sessionKey, message) == null) firstMessages.put(sessionKey, message); + }); + this.batchSize = resultSize; + return true; + } + + private static int calculateMessagesSize(GroupedMessageBatchToStore batch) { + return batch.messagesStream().mapToInt(StoredMessage::getSerializedSize).sum(); + } + + public Collection getSessionMessageBatches() throws CradleStorageException { + Map batches = new HashMap<>(); + for (StoredMessage message : getMessages()) { + SessionKey key = new SessionKey(message.getSessionAlias(), message.getDirection()); + MessageBatchToStore batch = batches.computeIfAbsent(key, k -> new MessageBatchToStore(maxBatchSize)); + + StoredMessageId msgId = new StoredMessageId( + message.getBookId(), + message.getSessionAlias(), + message.getDirection(), + message.getTimestamp(), + message.getSequence() + ); + + MessageToStoreBuilder builder = MessageToStore.builder() + .id(msgId) + .protocol(message.getProtocol()) + .content(message.getContent()); + + if (message.getMetadata() != null) + message.getMetadata().toMap().forEach(builder::metadata); + batch.addMessage(builder.build()); + } + + return batches.values(); + } + + + private void verifyBatch(GroupedMessageBatchToStore otherBatch) throws CradleStorageException { + if (!Objects.equals(bookId, otherBatch.getBookId())) + throw new CradleStorageException(String.format("Batch BookId-s differ. Current BookId is %s, other BookId is %s", bookId, otherBatch.getBookId())); + + if (this.getFirstTimestamp().isAfter(otherBatch.getFirstTimestamp()) || + this.getLastTimestamp().isAfter(otherBatch.getFirstTimestamp())) + throw new CradleStorageException( + String.format( + "Batches intersect by time. Current batch %s - %s, other batch %s - %s", + this.getFirstTimestamp(), + this.getLastTimestamp(), + otherBatch.getFirstTimestamp(), + otherBatch.getLastTimestamp() + )); + + for (SessionKey sessionKey : lastMessages.keySet()) { + StoredMessage otherFirstMessage = otherBatch.firstMessages.get(sessionKey); + if (otherFirstMessage == null) + continue; + StoredMessage thisLastMessage = this.lastMessages.get(sessionKey); + + if (thisLastMessage.getTimestamp().isAfter(otherFirstMessage.getTimestamp())) + throw new CradleStorageException(String.format("Batches are not ordered. Current last timestamp: %s; Other first timestamp: %s", + thisLastMessage.getTimestamp(), otherFirstMessage.getTimestamp() + )); + + if (thisLastMessage.getSequence() >= otherFirstMessage.getSequence()) { + throw new CradleStorageException(String.format("Batches are not ordered. Current last sequence number: %d; Other first sequence number: %d", + thisLastMessage.getSequence(), otherFirstMessage.getSequence() + )); + } + } + } + + private static class SessionKey { + final String sessionAlias; + final Direction direction; + + SessionKey(String sessionAlias, Direction direction) { + this.sessionAlias = sessionAlias; + this.direction = direction; + } + + @Override + public boolean equals(Object o) { + if (!(o instanceof SessionKey)) + return false; + + SessionKey that = (SessionKey) o; + if (!sessionAlias.equals(that.sessionAlias)) + return false; + return direction == that.direction; + } + + @Override + public int hashCode() { + int result = sessionAlias.hashCode(); + result = 31 * result + direction.hashCode(); + return result; + } + } } \ No newline at end of file diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java index 6ccae7689..c88be9168 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -32,21 +32,35 @@ public class MessageToStore implements CradleMessage { private MessageMetadata metadata; private int serializedSize; - MessageToStore(StoredMessageId id, String protocol, byte[] content) throws CradleStorageException { - this(id, protocol, content, null); - } - - MessageToStore(StoredMessageId id, String protocol, byte[] content, MessageMetadata metadata) throws CradleStorageException { + private MessageToStore(StoredMessageId id, String protocol, byte[] content, MessageMetadata metadata, int serializedSize) throws CradleStorageException { this.id = id; this.protocol = protocol; this.content = content; this.metadata = metadata; - MessageUtils.validateMessage(this); - this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); + if (serializedSize > 0) { + this.serializedSize = serializedSize; + } else { + MessageUtils.validateMessage(this); + this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); + } + } + + MessageToStore(StoredMessageId id, String protocol, byte[] content, MessageMetadata metadata) throws CradleStorageException { + this(id, protocol, content, metadata, -1); + } + + MessageToStore(StoredMessageId id, String protocol, byte[] content) throws CradleStorageException { + this(id, protocol, content, null); } public MessageToStore(MessageToStore copyFrom) throws CradleStorageException { - this(copyFrom.getId(), copyFrom.getProtocol(), copyFrom.getContent(), copyFrom.getMetadata() != null ? new MessageMetadata(copyFrom.getMetadata()) : null); + this( + copyFrom.getId(), + copyFrom.getProtocol(), + copyFrom.getContent(), + copyFrom.getMetadata() != null ? new MessageMetadata(copyFrom.getMetadata()) : null, + copyFrom.serializedSize + ); } public static MessageToStoreBuilder builder() { @@ -85,8 +99,9 @@ public void setMetadata(MessageMetadata metadata) { } public void addMetadata(String key, String value) { - if (metadata == null) + if (metadata == null) { metadata = new MessageMetadata(); + } metadata.add(key, value); this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); } @@ -94,12 +109,12 @@ public void addMetadata(String key, String value) { @Override public String toString() { - return new StringBuilder() - .append("MessageToStore{").append(System.lineSeparator()) - .append("id=").append(id).append(",").append(System.lineSeparator()) - .append("content=").append(Arrays.toString(content)).append(System.lineSeparator()) - .append("protocol=").append(protocol).append(",").append(System.lineSeparator()) - .append("metadata=").append(metadata).append(",").append(System.lineSeparator()) - .append("}").toString(); + return "MessageToStore{" + System.lineSeparator() + + "id=" + id + "," + System.lineSeparator() + + "content=" + Arrays.toString(content) + System.lineSeparator() + + "protocol=" + protocol + "," + System.lineSeparator() + + "metadata=" + metadata + "," + System.lineSeparator() + + "serializedSize=" + serializedSize + "," + System.lineSeparator() + + "}"; } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionType.java b/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionType.java index 667671bfc..8b1b2dcd6 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionType.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/utils/CompressionType.java @@ -16,6 +16,7 @@ package com.exactpro.cradle.utils; +import com.google.common.io.BaseEncoding; import net.jpountz.lz4.LZ4Compressor; import net.jpountz.lz4.LZ4Factory; import net.jpountz.lz4.LZ4FastDecompressor; @@ -24,13 +25,26 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.stream.Collectors; import java.util.zip.DataFormatException; import java.util.zip.Deflater; import java.util.zip.Inflater; public enum CompressionType { ZLIB { - public static final int BUFFER_SIZE = 4096; + private static final int BUFFER_SIZE = 4096; + // https://www.rfc-editor.org/rfc/rfc1950 + // 78 01 - No Compression/low + // 78 9C - Default Compression - used in our case + // 78 DA - Best Compression + private static final byte CMF = 0x78; + private static final byte FLG = (byte) 0x9C; + private final byte[] MAGIC_BYTES = new byte[] { CMF, FLG }; + + @Override + public byte[] getMagicBytes() { + return MAGIC_BYTES; + } @Override public byte[] compress(byte[] data) throws CompressException { @@ -70,43 +84,65 @@ public byte[] decompress(byte[] data) throws CompressException { LZ4 { private final LZ4Compressor COMPRESSOR = LZ4Factory.fastestInstance().fastCompressor(); private final LZ4FastDecompressor DECOMPRESSOR = LZ4Factory.fastestInstance().fastDecompressor(); + // LZ4 (ASCII) + private final byte[] MAGIC_BYTES = ByteBuffer.allocate(4).putInt(0x004C5A34).array(); + private final int PREFIX_SIZE = MAGIC_BYTES.length + Integer.BYTES; + + @Override + protected byte[] getMagicBytes() { + return MAGIC_BYTES; + } @Override public byte[] compress(byte[] data) { var length = data.length; var maxLength = COMPRESSOR.maxCompressedLength(length); - var compressed = new byte[Integer.BYTES + maxLength]; - var newLength = COMPRESSOR.compress(data, 0, length, compressed, Integer.BYTES, maxLength); + var compressed = new byte[PREFIX_SIZE + maxLength]; + var newLength = COMPRESSOR.compress(data, 0, length, compressed, PREFIX_SIZE, maxLength); var buffer = ByteBuffer.wrap(compressed); - buffer.putInt(0, length); - return Arrays.copyOf(compressed, Integer.BYTES + newLength); + buffer.put(MAGIC_BYTES, 0, MAGIC_BYTES.length); + buffer.putInt(MAGIC_BYTES.length, length); + return Arrays.copyOf(compressed, PREFIX_SIZE + newLength); } @Override public byte[] decompress(byte[] data) { - return DECOMPRESSOR.decompress(data, Integer.BYTES, ByteBuffer.wrap(data).getInt()); + return DECOMPRESSOR.decompress(data, MAGIC_BYTES.length + Integer.BYTES, ByteBuffer.wrap(data).getInt(MAGIC_BYTES.length)); } }; public abstract byte[] compress(byte[] data) throws CompressException; - public abstract byte[] decompress(byte[] data) throws CompressException; - // https://www.rfc-editor.org/rfc/rfc1950 - // 78 01 - No Compression/low - // 78 9C - Default Compression - used in our case - // 78 DA - Best Compression - public static final byte ZLIB_CMF = 0x78; - public static final byte ZLIB_FLG = (byte) 0x9C; + public boolean isDecompressable(byte[] data) { + byte[] magicBytes = getMagicBytes(); + if (magicBytes.length > data.length) { + return false; + } + for (int i = 0; i < magicBytes.length; i++) { + if (magicBytes[i] != data[i]) { + return false; + } + } + return true; + } + + protected abstract byte[] getMagicBytes(); public static byte[] decompressData(byte[] data) throws CompressException { - if (data.length < 2) { - throw new CompressException("Data too short. Compression format is undetected"); + if (data == null || data.length == 0) { + throw new CompressException("Data is empty or null"); } - if (data[0] == ZLIB_CMF && data[1] == ZLIB_FLG) { + if (ZLIB.isDecompressable(data)) { return ZLIB.decompress(data); - } else { + } else if (LZ4.isDecompressable(data)) { return LZ4.decompress(data); + } else { + BaseEncoding encoder = BaseEncoding.base16(); + throw new CompressException("Compression format is undetected. Decoded data: " + encoder.encode(data, 0, Math.min(10, data.length)) + + ", supported formats: " + Arrays.stream(CompressionType.values()) + .map(type -> type.name() + ":[" + encoder.encode(type.getMagicBytes()) + "]") + .collect(Collectors.joining(","))); } } } diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java index 67ffd4e48..715b239a5 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageBatchToStoreJoinTest.java @@ -102,6 +102,17 @@ public void testAddBatchLessThanLimit() throws CradleStorageException { assertEquals(first.getGroup(), groupName); } + @Test + public void testAddBatchMoreThanLimitBySizeToEmptyBatch() throws CradleStorageException { + GroupedMessageBatchToStore empty = new GroupedMessageBatchToStore(groupName, MAX_SIZE / 4); + GroupedMessageBatchToStore second = createBatch(bookId, "test", 5, Direction.FIRST, Instant.EPOCH, 1, MAX_SIZE / 2, groupName, protocol); + + long sizeBefore = empty.getBatchSize(); + assertFalse(empty.addBatch(second)); + assertEquals(empty.getMessageCount(), 0); + assertEquals(empty.getBatchSize(), sizeBefore); + } + @Test public void testAddBatchMoreThanLimitBySize() throws CradleStorageException { GroupedMessageBatchToStore first = createBatch(bookId, "test", 0, Direction.FIRST, Instant.EPOCH, 1, MAX_SIZE / 2, groupName, protocol); diff --git a/cradle-core/src/test/java/com/exactpro/cradle/utils/CompressionTypeTest.java b/cradle-core/src/test/java/com/exactpro/cradle/utils/CompressionTypeTest.java index ac3bdf586..4744a6147 100644 --- a/cradle-core/src/test/java/com/exactpro/cradle/utils/CompressionTypeTest.java +++ b/cradle-core/src/test/java/com/exactpro/cradle/utils/CompressionTypeTest.java @@ -44,4 +44,9 @@ public void compressDecompressZL4() throws CompressException { assertEquals(bytes, LZ4.decompress(compressed), "Decompress via " + LZ4); assertEquals(bytes, CompressionType.decompressData(compressed), "Decompress via the decompressData method"); } + + @Test(expectedExceptions = { CompressException.class }, expectedExceptionsMessageRegExp = "Compression format is undetected\\. Decoded data: 12, supported formats: ZLIB:\\[789C],LZ4:\\[004C5A34]") + public void decompressionFailure() throws CompressException { + CompressionType.decompressData(new byte[] { 0x12 }); + } } From 5d1e60e918293f29f19c88b9b9307b48ae0a1222 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 12 Apr 2023 16:42:31 +0400 Subject: [PATCH 18/31] Made MessageToStore immutable --- .../cradle/messages/MessageToStore.java | 22 ++----------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java index c88be9168..429920e46 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/MessageToStore.java @@ -29,8 +29,8 @@ public class MessageToStore implements CradleMessage { private final StoredMessageId id; private final String protocol; private final byte[] content; - private MessageMetadata metadata; - private int serializedSize; + private final MessageMetadata metadata; + private final int serializedSize; private MessageToStore(StoredMessageId id, String protocol, byte[] content, MessageMetadata metadata, int serializedSize) throws CradleStorageException { this.id = id; @@ -49,10 +49,6 @@ private MessageToStore(StoredMessageId id, String protocol, byte[] content, Mess this(id, protocol, content, metadata, -1); } - MessageToStore(StoredMessageId id, String protocol, byte[] content) throws CradleStorageException { - this(id, protocol, content, null); - } - public MessageToStore(MessageToStore copyFrom) throws CradleStorageException { this( copyFrom.getId(), @@ -93,20 +89,6 @@ public int getSerializedSize() { return serializedSize; } - public void setMetadata(MessageMetadata metadata) { - this.metadata = metadata; - this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); - } - - public void addMetadata(String key, String value) { - if (metadata == null) { - metadata = new MessageMetadata(); - } - metadata.add(key, value); - this.serializedSize = MessagesSizeCalculator.calculateMessageSizeInBatch(this); - } - - @Override public String toString() { return "MessageToStore{" + System.lineSeparator() + From c761a84ddb424cdaa19fb24e99002e40b299118d Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Fri, 21 Apr 2023 16:28:38 +0400 Subject: [PATCH 19/31] Added getter for max event / message size to Entities Factory --- .../java/com/exactpro/cradle/CradleEntitiesFactory.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleEntitiesFactory.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleEntitiesFactory.java index e41a7938e..202959a94 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleEntitiesFactory.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleEntitiesFactory.java @@ -68,4 +68,12 @@ public TestEventBatchToStoreBuilder testEventBatchBuilder() { return new TestEventBatchToStoreBuilder(maxTestEventBatchSize); } + + public int getMaxMessageBatchSize() { + return maxMessageBatchSize; + } + + public int getMaxTestEventBatchSize() { + return maxTestEventBatchSize; + } } From dda4d35754c5b9e2def82d764ddea47e93a096ce Mon Sep 17 00:00:00 2001 From: Oleg Smirnov Date: Mon, 24 Apr 2023 10:35:52 +0400 Subject: [PATCH 20/31] =?UTF-8?q?[TH2-4883]=20Correct=20iteration=20over?= =?UTF-8?q?=20pages=20in=20group=20search=20in=20reversed=20o=E2=80=A6=20(?= =?UTF-8?q?#231)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [TH2-4883] Correct iteration over pages in group search in reversed order * [TH2-4883] Add time validation into abstract filter * Update copyright Co-authored-by: Nikita Smirnov <46124551+Nikita-Smirnov-Exactpro@users.noreply.github.com> --------- Co-authored-by: Nikita Smirnov <46124551+Nikita-Smirnov-Exactpro@users.noreply.github.com> --- .../cassandra/dao/CommonQueryProvider.java | 40 +- .../CassandraGroupedMessageFilter.java | 28 +- .../GroupedMessageIteratorProvider.java | 44 +- .../com/exactpro/cradle/CradleStorage.java | 542 +++++++++--------- .../cradle/filters/AbstractFilter.java | 69 +-- .../cradle/filters/AbstractFilterBuilder.java | 50 +- .../cradle/messages/GroupedMessageFilter.java | 42 +- .../GroupedMessageFilterBuilderTest.java | 109 ++++ 8 files changed, 513 insertions(+), 411 deletions(-) create mode 100644 cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageFilterBuilderTest.java diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/CommonQueryProvider.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/CommonQueryProvider.java index b091aae57..5fe42df17 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/CommonQueryProvider.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/CommonQueryProvider.java @@ -28,41 +28,45 @@ import com.datastax.oss.driver.api.mapper.entity.EntityHelper; import com.datastax.oss.driver.api.querybuilder.select.Select; import com.exactpro.cradle.cassandra.retries.SelectQueryExecutor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -public class CommonQueryProvider -{ +public class CommonQueryProvider { + private static final Logger LOGGER = LoggerFactory.getLogger(CommonQueryProvider.class); private final CqlSession session; private final EntityHelper helper; - - public CommonQueryProvider(MapperContext context, EntityHelper helper) - { + + public CommonQueryProvider(MapperContext context, EntityHelper helper) { this.session = context.getSession(); this.helper = helper; } - - public CompletableFuture> getByFilter(CassandraFilter filter, + + public CompletableFuture> getByFilter( + CassandraFilter filter, SelectQueryExecutor selectExecutor, String queryInfo, - Function attributes) - { + Function attributes + ) { Select select = createQuery(filter); PreparedStatement ps = session.prepare(select.build()); BoundStatement bs = bindParameters(ps, filter, attributes); return selectExecutor.executeMultiRowResultQuery( () -> session.executeAsync(bs).toCompletableFuture(), helper::get, queryInfo); } - - - private Select createQuery(CassandraFilter filter) - { + + + private Select createQuery(CassandraFilter filter) { Select select = helper.selectStart(); - if (filter != null) + if (filter != null) { select = filter.addConditions(select); + } + LOGGER.trace("Executing query {}", select); return select; } - - private BoundStatement bindParameters(PreparedStatement ps, CassandraFilter filter, - Function attributes) - { + + private BoundStatement bindParameters( + PreparedStatement ps, CassandraFilter filter, + Function attributes + ) { BoundStatementBuilder builder = ps.boundStatementBuilder(); builder = attributes.apply(builder); if (filter != null) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/CassandraGroupedMessageFilter.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/CassandraGroupedMessageFilter.java index 6b84a6854..c3fd0b38b 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/CassandraGroupedMessageFilter.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/CassandraGroupedMessageFilter.java @@ -26,6 +26,7 @@ import com.exactpro.cradle.filters.FilterForLess; import java.time.Instant; +import java.util.StringJoiner; import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.bindMarker; import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.*; @@ -57,8 +58,8 @@ public CassandraGroupedMessageFilter(String book, @Override public Select addConditions(Select select) { select = select - .whereColumn(FIELD_BOOK).isEqualTo(bindMarker()) - .whereColumn(FIELD_PAGE).isEqualTo(bindMarker()) + .whereColumn(FIELD_BOOK).isEqualTo(bindMarker()) + .whereColumn(FIELD_PAGE).isEqualTo(bindMarker()) .whereColumn(FIELD_ALIAS_GROUP).isEqualTo(bindMarker()); if (messageTimeFrom != null) @@ -84,8 +85,8 @@ public Select addConditions(Select select) { @Override public BoundStatementBuilder bindParameters(BoundStatementBuilder builder) { builder = builder - .setString(FIELD_BOOK, book) - .setString(FIELD_PAGE, page) + .setString(FIELD_BOOK, book) + .setString(FIELD_PAGE, page) .setString(FIELD_ALIAS_GROUP, groupName); if (messageTimeFrom != null) @@ -96,9 +97,9 @@ public BoundStatementBuilder bindParameters(BoundStatementBuilder builder) { return builder; } - public String getBook() { - return book; - } + public String getBook() { + return book; + } public String getPage() { return page; @@ -119,4 +120,17 @@ public FilterForGreater getMessageTimeFrom() { public FilterForLess getMessageTimeTo() { return messageTimeTo; } + + @Override + public String toString() { + return new StringJoiner(", ", CassandraGroupedMessageFilter.class.getSimpleName() + "[", "]") + .add("book='" + book + "'") + .add("page='" + page + "'") + .add("groupName='" + groupName + "'") + .add("limit=" + limit) + .add("messageTimeFrom " + messageTimeFrom) + .add("messageTimeTo " + messageTimeTo) + .add("order=" + order) + .toString(); + } } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java index 86a785d51..73955ce2a 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Instant; import java.util.Iterator; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; @@ -43,7 +44,7 @@ public class GroupedMessageIteratorProvider extends IteratorProvider { public static final Logger logger = LoggerFactory.getLogger(GroupedMessageIteratorProvider.class); - + private final GroupedMessageBatchOperator op; private final GroupedMessageBatchEntityConverter converter; private final BookInfo book; @@ -64,8 +65,7 @@ public GroupedMessageIteratorProvider(String requestInfo, ExecutorService composingService, SelectQueryExecutor selectQueryExecutor, Function readAttrs, - Order order) throws CradleStorageException - { + Order order) throws CradleStorageException { super(requestInfo); this.op = operators.getGroupedMessageBatchOperator(); @@ -86,11 +86,10 @@ public GroupedMessageIteratorProvider(String requestInfo, this.cassandraFilter = createInitialFilter(filter); } - private CassandraGroupedMessageFilter createInitialFilter(GroupedMessageFilter filter) - { + private CassandraGroupedMessageFilter createInitialFilter(GroupedMessageFilter filter) { return new CassandraGroupedMessageFilter( book.getId().getName(), - firstPage.getId().getName(), + getFirstPage().getId().getName(), filter.getGroupName(), filter.getFrom(), filter.getTo(), @@ -98,13 +97,12 @@ private CassandraGroupedMessageFilter createInitialFilter(GroupedMessageFilter f filter.getLimit()); } - protected CassandraGroupedMessageFilter createNextFilter(CassandraGroupedMessageFilter prevFilter, int updatedLimit) - { + protected CassandraGroupedMessageFilter createNextFilter(CassandraGroupedMessageFilter prevFilter, int updatedLimit) { PageInfo prevPage = book.getPage(new PageId(book.getId(), prevFilter.getPage())); - if (prevPage.equals(lastPage)) + if (prevPage.equals(getLastPage())) return null; - PageInfo nextPage = book.getNextPage(prevPage.getStarted()); + PageInfo nextPage = getNextPage(prevPage.getStarted()); return new CassandraGroupedMessageFilter( book.getId().getName(), @@ -117,12 +115,10 @@ protected CassandraGroupedMessageFilter createNextFilter(CassandraGroupedMessage } @Override - public CompletableFuture> nextIterator() - { + public CompletableFuture> nextIterator() { if (cassandraFilter == null) return CompletableFuture.completedFuture(null); - if (limit > 0 && returned.get() >= limit) - { + if (limit > 0 && returned.get() >= limit) { logger.debug("Filtering interrupted because limit for records to return ({}) is reached ({})", limit, returned); return CompletableFuture.completedFuture(null); } @@ -133,11 +129,29 @@ public CompletableFuture> nextIterator() { PageId pageId = new PageId(book.getId(), cassandraFilter.getPage()); // Updated limit should be smaller, since we already got entities from previous batch - cassandraFilter = createNextFilter(cassandraFilter, Math.max(limit - returned.get(),0)); + cassandraFilter = createNextFilter(cassandraFilter, Math.max(limit - returned.get(), 0)); return new ConvertingPagedIterator<>(resultSet, selectQueryExecutor, 0, new AtomicInteger(), entity -> MessagesWorker.mapGroupedMessageBatchEntity(pageId, entity), converter::getEntity, "fetch next page of message batches"); }, composingService) .thenApplyAsync(it -> new FilteredGroupedMessageBatchIterator(it, filter, limit, returned), composingService); } + + private boolean isDirectOrder() { + return order == Order.DIRECT; + } + + private PageInfo getFirstPage() { + return isDirectOrder() ? firstPage : lastPage; + } + + private PageInfo getLastPage() { + return isDirectOrder() ? lastPage : firstPage; + } + + private PageInfo getNextPage(Instant currentPageStart) { + return isDirectOrder() + ? book.getNextPage(currentPageStart) + : book.getPreviousPage(currentPageStart); + } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index 1f81a5e16..04f9994f0 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -66,8 +66,7 @@ public abstract class CradleStorage protected final CradleEntitiesFactory entitiesFactory; public CradleStorage(ExecutorService composingService, int composingServiceThreads, int maxMessageBatchSize, - int maxTestEventBatchSize) throws CradleStorageException - { + int maxTestEventBatchSize) throws CradleStorageException { if (composingService == null) { ownedComposingService = true; this.composingService = Executors.newFixedThreadPool(composingServiceThreads, THREAD_FACTORY); @@ -76,80 +75,108 @@ public CradleStorage(ExecutorService composingService, int composingServiceThrea ownedComposingService = false; this.composingService = composingService; } - + entitiesFactory = new CradleEntitiesFactory(maxMessageBatchSize, maxTestEventBatchSize); } - - public CradleStorage() throws CradleStorageException - { + + public CradleStorage() throws CradleStorageException { this(null, DEFAULT_COMPOSING_SERVICE_THREADS, DEFAULT_MAX_MESSAGE_BATCH_SIZE, DEFAULT_MAX_TEST_EVENT_BATCH_SIZE); } - + protected abstract void doInit(boolean prepareStorage) throws CradleStorageException; + protected abstract BookCache getBookCache (); + protected abstract void doDispose() throws CradleStorageException; protected abstract Collection doListBooks (); + protected abstract void doAddBook(BookToAdd newBook, BookId bookId) throws IOException; + protected abstract void doAddPages(BookId bookId, List pages, PageInfo lastPage) throws CradleStorageException, IOException; + protected abstract Collection doLoadPages(BookId bookId) throws CradleStorageException, IOException; + protected abstract Collection doGetAllPages(BookId bookId) throws CradleStorageException; + protected abstract void doRemovePage(PageInfo page) throws CradleStorageException, IOException; - - + + protected abstract void doStoreMessageBatch(MessageBatchToStore batch, PageInfo page) throws IOException, CradleStorageException; + protected abstract void doStoreGroupedMessageBatch(GroupedMessageBatchToStore batch, PageInfo page) throws IOException; + protected abstract CompletableFuture doStoreMessageBatchAsync(MessageBatchToStore batch, PageInfo page) throws IOException, CradleStorageException; + protected abstract CompletableFuture doStoreGroupedMessageBatchAsync(GroupedMessageBatchToStore batch, PageInfo page) throws IOException, CradleStorageException; - - + + protected abstract void doStoreTestEvent(TestEventToStore event, PageInfo page) throws IOException, CradleStorageException; + protected abstract CompletableFuture doStoreTestEventAsync(TestEventToStore event, PageInfo page) throws IOException, CradleStorageException; + protected abstract void doUpdateParentTestEvents(TestEventToStore event) throws IOException; + protected abstract CompletableFuture doUpdateParentTestEventsAsync(TestEventToStore event); + protected abstract void doUpdateEventStatus(StoredTestEvent event, boolean success) throws IOException; + protected abstract CompletableFuture doUpdateEventStatusAsync(StoredTestEvent event, boolean success); - - + + protected abstract StoredMessage doGetMessage(StoredMessageId id, PageId pageId) throws IOException, CradleStorageException; + protected abstract CompletableFuture doGetMessageAsync(StoredMessageId id, PageId pageId) throws CradleStorageException; + protected abstract StoredMessageBatch doGetMessageBatch(StoredMessageId id, PageId pageId) throws IOException, CradleStorageException; + protected abstract CompletableFuture doGetMessageBatchAsync(StoredMessageId id, PageId pageId) throws CradleStorageException; - - protected abstract CradleResultSet doGetMessages(MessageFilter filter, BookInfo book) + + protected abstract CradleResultSet doGetMessages(MessageFilter filter, BookInfo book) throws IOException, CradleStorageException; + protected abstract CompletableFuture> doGetMessagesAsync(MessageFilter filter, BookInfo book) throws CradleStorageException; - protected abstract CradleResultSet doGetMessageBatches(MessageFilter filter, BookInfo book) + + protected abstract CradleResultSet doGetMessageBatches(MessageFilter filter, BookInfo book) throws IOException, CradleStorageException; + protected abstract CradleResultSet doGetGroupedMessageBatches(GroupedMessageFilter filter, BookInfo book) throws IOException, CradleStorageException; + protected abstract CompletableFuture> doGetMessageBatchesAsync(MessageFilter filter, - BookInfo book) throws CradleStorageException; + BookInfo book) throws CradleStorageException; + protected abstract CompletableFuture> doGetGroupedMessageBatchesAsync(GroupedMessageFilter filter, - BookInfo book) throws CradleStorageException; - + BookInfo book) throws CradleStorageException; + protected abstract long doGetLastSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException; + protected abstract long doGetFirstSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException; + protected abstract Collection doGetSessionAliases(BookId bookId) throws IOException, CradleStorageException; + protected abstract Collection doGetGroups(BookId bookId) throws IOException, CradleStorageException; - - + + protected abstract StoredTestEvent doGetTestEvent(StoredTestEventId id, PageId pageId) throws IOException, CradleStorageException; + protected abstract CompletableFuture doGetTestEventAsync(StoredTestEventId ids, PageId pageId) throws CradleStorageException; - protected abstract CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) + + protected abstract CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) throws IOException, CradleStorageException; - protected abstract CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) + + protected abstract CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) throws CradleStorageException; - + protected abstract Collection doGetScopes(BookId bookId) throws IOException, CradleStorageException; protected abstract CompletableFuture> doGetMessageCountersAsync(BookId bookId, @@ -157,6 +184,7 @@ protected abstract CompletableFuture> doGetMessag Direction direction, FrameType frameType, Interval interval) throws CradleStorageException; + protected abstract CradleResultSet doGetMessageCounters(BookId bookId, String sessionAlias, Direction direction, @@ -167,6 +195,7 @@ protected abstract CompletableFuture> doGetCounte EntityType entityType, FrameType frameType, Interval interval) throws CradleStorageException; + protected abstract CradleResultSet doGetCounters(BookId bookId, EntityType entityType, FrameType frameType, @@ -217,13 +246,12 @@ protected abstract CradleResultSet doGetSessionGroups(BookId bookId, * @throws CradleStorageException if storage initialization failed * @throws IOException if data reading or creation of storage failed */ - public void init(boolean prepareStorage) throws CradleStorageException, IOException - { + public void init(boolean prepareStorage) throws CradleStorageException, IOException { if (initialized) return; - + logger.info("Initializing storage"); - + doInit(prepareStorage); // In case it did not get initialized in doInit if (bookManager == null) { @@ -232,45 +260,42 @@ public void init(boolean prepareStorage) throws CradleStorageException, IOExcept initialized = true; logger.info("Storage initialized"); } - + /** * IntervalsWorker is used to work with Crawler intervals * @return instance of IntervalsWorker */ public abstract IntervalsWorker getIntervalsWorker(); - - + + /** * Disposes resources occupied by storage which means closing of opened connections, flushing all buffers, etc. * @throws CradleStorageException if there was error during storage disposal, which may mean issue with data flushing, unexpected connection break, etc. */ - public final void dispose() throws CradleStorageException - { + public final void dispose() throws CradleStorageException { if (disposed) return; - + logger.info("Disposing storage"); - - if (ownedComposingService) - { + + if (ownedComposingService) { logger.info("Shutting down composing service..."); composingService.shutdownNow(); } - + doDispose(); disposed = true; logger.info("Storage disposed"); } - + /** * @return true if storage is already disposed and false if it is not disposed, including the case when disposal failed with error */ - public final boolean isDisposed() - { + public final boolean isDisposed() { return disposed; } - - + + /** * Creates new book and adds it to storage, adding page with given name to newly created book * @param book information about book to add and its first page @@ -278,15 +303,14 @@ public final boolean isDisposed() * @throws CradleStorageException if the book is already present * @throws IOException if book data writing failed */ - public BookInfo addBook(BookToAdd book) throws CradleStorageException, IOException - { + public BookInfo addBook(BookToAdd book) throws CradleStorageException, IOException { BookPagesNamesChecker.validateBookName(book.getName()); BookId id = new BookId(book.getName()); logger.info("Adding book '{}' to storage", id); if (checkBook(id)) throw new CradleStorageException("Book '"+id+"' is already present in storage"); - + doAddBook(book, id); BookInfo newBook = new BookInfo(id, book.getFullName(), book.getDesc(), book.getCreated(), null); getBookCache().updateCachedBook(newBook); @@ -308,13 +332,12 @@ public Collection listBooks () { /** * @return collection of books currently available in storage */ - public Collection getBooks() - { + public Collection getBooks() { return Collections.unmodifiableCollection(getBookCache().getCachedBooks()); } - + /** - * Adds to given book the new page, started at current timestamp. + * Adds to given book the new page, started at current timestamp. * Last page of the book will be marked as ended at timestamp of new page start * @param bookId ID of the book where to add the page * @param pageName name of new page @@ -324,13 +347,12 @@ public Collection getBooks() * @throws CradleStorageException if given bookId is unknown or page with given name already exists in this book * @throws IOException if page data writing failed */ - public BookInfo addPage(BookId bookId, String pageName, Instant pageStart, String pageComment) throws CradleStorageException, IOException - { + public BookInfo addPage(BookId bookId, String pageName, Instant pageStart, String pageComment) throws CradleStorageException, IOException { return addPages(bookId, Collections.singletonList(new PageToAdd(pageName, pageStart, pageComment))); } - + /** - * Adds new pages to given book. + * Adds new pages to given book. * Last page of the book will be marked as ended at start timestamp of the first page being added * @param bookId ID of the book where to add the page * @param pages to add @@ -338,21 +360,20 @@ public BookInfo addPage(BookId bookId, String pageName, Instant pageStart, Strin * @throws CradleStorageException if given bookId is unknown, page to add already exists or new pages are not in ascending order * @throws IOException if page data writing failed */ - public BookInfo addPages(BookId bookId, List pages) throws CradleStorageException, IOException - { + public BookInfo addPages(BookId bookId, List pages) throws CradleStorageException, IOException { logger.info("Adding pages {} to book '{}'", pages, bookId); - + BookInfo book = refreshPages(bookId); if (pages == null || pages.isEmpty()) return book; - + List toAdd = checkPages(pages, book); - + PageInfo bookLastPage = book.getLastPage(); PageInfo endedPage = null; PageInfo lastPageToAdd = !toAdd.isEmpty() ? toAdd.get(toAdd.size()-1) : null; - /* + /* If last page of toAdd list is after current last we need to finish current last page in book */ @@ -361,38 +382,34 @@ public BookInfo addPages(BookId bookId, List pages) throws CradleStor && lastPageToAdd != null && lastPageToAdd.getStarted().isAfter(bookLastPage.getStarted())) { - endedPage = PageInfo.ended(bookLastPage, toAdd.get(0).getStarted()); + endedPage = PageInfo.ended(bookLastPage, toAdd.get(0).getStarted()); } - - try - { + + try { doAddPages(bookId, toAdd, endedPage); - } - catch (IOException e) - { + } catch (IOException e) { //Need to refresh book's pages to make user able to see what was the reason of failure, e.g. new page was actually present refreshPages(bookId); throw e; } - + if (endedPage != null) book.addPage(endedPage); //Replacing last page with ended one, i.e. updating last page info for (PageInfo newPage : toAdd) book.addPage(newPage); - + return book; } - + /** - * Refreshes pages information of given book, loading actual data from storage. + * Refreshes pages information of given book, loading actual data from storage. * Use this method to refresh Cradle API internal book cache when new pages were added to the book or removed outside of the application * @param bookId ID of the book whose pages to refresh * @return refreshed book information * @throws CradleStorageException if given bookId is unknown * @throws IOException if page data reading failed */ - public BookInfo refreshPages(BookId bookId) throws CradleStorageException, IOException - { + public BookInfo refreshPages(BookId bookId) throws CradleStorageException, IOException { logger.info("Refreshing pages of book '{}'", bookId); BookInfo book = getBookCache().getBook(bookId); Collection pages = doLoadPages(bookId); @@ -434,7 +451,7 @@ public BookInfo refreshBook (String name) throws CradleStorageException { return bookInfo; } - + /** * Removes page with given ID, deleting all messages and test events stored within that page * @param pageId ID of page to remove @@ -442,13 +459,12 @@ public BookInfo refreshBook (String name) throws CradleStorageException { * @throws CradleStorageException if given page ID or its book is unknown or the page is currently the active one * @throws IOException if page data removal failed */ - public BookInfo removePage(PageId pageId) throws CradleStorageException, IOException - { + public BookInfo removePage(PageId pageId) throws CradleStorageException, IOException { logger.info("Removing page '{}'", pageId); - + BookId bookId = pageId.getBookId(); BookInfo book = refreshPages(bookId); - + String pageName = pageId.getName(); PageInfo page = book.getPage(pageId); if (page == null) @@ -458,17 +474,16 @@ public BookInfo removePage(PageId pageId) throws CradleStorageException, IOExcep logger.info("Page '{}' has been removed", pageId); return book; } - - + + /** * @return factory to create message and test event batches that conform with storage settings */ - public CradleEntitiesFactory getEntitiesFactory() - { + public CradleEntitiesFactory getEntitiesFactory() { return entitiesFactory; } - - + + /** * Writes data about given message batch to current page * @param batch data to write @@ -476,8 +491,7 @@ public CradleEntitiesFactory getEntitiesFactory() * @throws CradleStorageException if given parameters are invalid */ @Deprecated - public final void storeMessageBatch(MessageBatchToStore batch) throws IOException, CradleStorageException - { + public final void storeMessageBatch(MessageBatchToStore batch) throws IOException, CradleStorageException { StoredMessageId id = batch.getId(); logger.debug("Storing message batch {}", id); PageInfo page = findPage(id.getBookId(), id.getTimestamp()); @@ -485,10 +499,9 @@ public final void storeMessageBatch(MessageBatchToStore batch) throws IOExceptio logger.debug("Message batch {} has been stored", id); } - + public final void storeGroupedMessageBatch(GroupedMessageBatchToStore batch) - throws CradleStorageException, IOException - { + throws CradleStorageException, IOException { String groupName = batch.getGroup(); String id = String.format("%s:%s", batch.getBookId(), batch.getFirstTimestamp().toString()); logger.debug("Storing message batch {} grouped by {}", id, groupName); @@ -500,8 +513,8 @@ public final void storeGroupedMessageBatch(GroupedMessageBatchToStore batch) logger.debug("Message batch {} grouped by {} has been stored", id, groupName); } - - + + /** * Asynchronously writes data about given message batch to current page * @param batch data to write @@ -511,18 +524,17 @@ public final void storeGroupedMessageBatch(GroupedMessageBatchToStore batch) */ @Deprecated public final CompletableFuture storeMessageBatchAsync(MessageBatchToStore batch) - throws CradleStorageException, IOException - { + throws CradleStorageException, IOException { StoredMessageId id = batch.getId(); logger.debug("Storing message batch {} asynchronously", id); PageInfo page = findPage(id.getBookId(), id.getTimestamp()); CompletableFuture result = doStoreMessageBatchAsync(batch, page); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while storing message batch "+id+" asynchronously", error); - else - logger.debug("Message batch {} has been stored asynchronously", id); - }, composingService); + if (error != null) + logger.error("Error while storing message batch "+id+" asynchronously", error); + else + logger.debug("Message batch {} has been stored asynchronously", id); + }, composingService); return result; } @@ -578,8 +590,7 @@ List> paginateBatch(GroupedMessageBat * @throws CradleStorageException if given parameters are invalid */ public final CompletableFuture storeGroupedMessageBatchAsync(GroupedMessageBatchToStore batch) - throws CradleStorageException - { + throws CradleStorageException { String groupName = batch.getGroup(); if (groupName == null) throw new CradleStorageException("'groupName' is required parameter and can not be null"); @@ -643,15 +654,15 @@ TestEventToStore alignEventTimestampsToPage(TestEventToStore event, PageInfo pag for (var e : batch.getTestEvents()) { TestEventSingleToStore newEvent = new TestEventSingleToStoreBuilder() - .id(idMappings.getOrDefault(e.getId(), e.getId())) - .name(e.getName()) - .parentId(idMappings.getOrDefault(e.getParentId(), e.getParentId())) - .type(e.getType()) - .endTimestamp(e.getEndTimestamp()) - .success(e.isSuccess()) - .messages(e.getMessages()) - .content(e.getContent()) - .build(); + .id(idMappings.getOrDefault(e.getId(), e.getId())) + .name(e.getName()) + .parentId(idMappings.getOrDefault(e.getParentId(), e.getParentId())) + .type(e.getType()) + .endTimestamp(e.getEndTimestamp()) + .success(e.isSuccess()) + .messages(e.getMessages()) + .content(e.getContent()) + .build(); newBatch.addTestEvent(newEvent); } @@ -664,15 +675,15 @@ TestEventToStore alignEventTimestampsToPage(TestEventToStore event, PageInfo pag * @throws IOException if data writing failed * @throws CradleStorageException if given parameters are invalid */ - public final void storeTestEvent(TestEventToStore event) throws IOException, CradleStorageException { + public final void storeTestEvent(TestEventToStore event) throws IOException, CradleStorageException { StoredTestEventId id = event.getId(); logger.debug("Storing test event {}", id); PageInfo page = findPage(id.getBookId(), id.getStartTimestamp()); - + TestEventUtils.validateTestEvent(event, getBookCache().getBook(id.getBookId())); final TestEventToStore alignedEvent = alignEventTimestampsToPage(event, page); - + doStoreTestEvent(alignedEvent, page); logger.debug("Test event {} has been stored", id); if (alignedEvent.getParentId() != null) { @@ -681,7 +692,7 @@ public final void storeTestEvent(TestEventToStore event) throws IOException, Cra logger.debug("Parents of test event {} have been updated", id); } } - + /** * Asynchronously writes data about given test event to current page * @param event data to write @@ -694,35 +705,35 @@ public final CompletableFuture storeTestEventAsync(TestEventToStore event) StoredTestEventId id = event.getId(); logger.debug("Storing test event {} asynchronously", id); PageInfo page = findPage(id.getBookId(), id.getStartTimestamp()); - + TestEventUtils.validateTestEvent(event, getBookCache().getBook(id.getBookId())); final TestEventToStore alignedEvent = alignEventTimestampsToPage(event, page); - + CompletableFuture result = doStoreTestEventAsync(alignedEvent, page); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while storing test event "+id+" asynchronously", error); - else - logger.debug("Test event {} has been stored asynchronously", id); - }, composingService); - + if (error != null) + logger.error("Error while storing test event "+id+" asynchronously", error); + else + logger.debug("Test event {} has been stored asynchronously", id); + }, composingService); + if (alignedEvent.getParentId() == null) return result; - + return result.thenComposeAsync(r -> { logger.debug("Updating parents of test event {} asynchronously", id); CompletableFuture result2 = doUpdateParentTestEventsAsync(alignedEvent); result2.whenCompleteAsync((r2, error) -> { - if (error != null) - logger.error("Error while updating parents of test event "+id+" asynchronously", error); - else - logger.debug("Parents of test event {} have been updated asynchronously", alignedEvent.getId()); - }, composingService); + if (error != null) + logger.error("Error while updating parents of test event "+id+" asynchronously", error); + else + logger.debug("Parents of test event {} have been updated asynchronously", alignedEvent.getId()); + }, composingService); return result2; }, composingService); } - - + + /** * Retrieves message data stored under given ID * @param id of stored message to retrieve @@ -730,36 +741,34 @@ public final CompletableFuture storeTestEventAsync(TestEventToStore event) * @throws IOException if message data retrieval failed * @throws CradleStorageException if given parameter is invalid */ - public final StoredMessage getMessage(StoredMessageId id) throws IOException, CradleStorageException - { + public final StoredMessage getMessage(StoredMessageId id) throws IOException, CradleStorageException { logger.debug("Getting message {}", id); PageId pageId = findPage(id.getBookId(), id.getTimestamp()).getId(); StoredMessage result = doGetMessage(id, pageId); logger.debug("Message {} got from page {}", id, pageId); return result; } - + /** * Asynchronously retrieves message data stored under given ID * @param id of stored message to retrieve * @return future to obtain data of stored message * @throws CradleStorageException if given parameter is invalid */ - public final CompletableFuture getMessageAsync(StoredMessageId id) throws CradleStorageException - { + public final CompletableFuture getMessageAsync(StoredMessageId id) throws CradleStorageException { logger.debug("Getting message {} asynchronously", id); PageId pageId = findPage(id.getBookId(), id.getTimestamp()).getId(); CompletableFuture result = doGetMessageAsync(id, pageId); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting message "+id+" from page "+pageId+" asynchronously", error); - else - logger.debug("Message {} from page {} got asynchronously", id, pageId); - }, composingService); + if (error != null) + logger.error("Error while getting message "+id+" from page "+pageId+" asynchronously", error); + else + logger.debug("Message {} from page {} got asynchronously", id, pageId); + }, composingService); return result; } - - + + /** * Retrieves the batch of messages where message with given ID is stored * @param id of stored message whose batch to retrieve @@ -767,35 +776,33 @@ public final CompletableFuture getMessageAsync(StoredMessageId id * @throws IOException if batch data retrieval failed * @throws CradleStorageException if given parameter is invalid */ - public final StoredMessageBatch getMessageBatch(StoredMessageId id) throws IOException, CradleStorageException - { + public final StoredMessageBatch getMessageBatch(StoredMessageId id) throws IOException, CradleStorageException { logger.debug("Getting message batch by message ID {}", id); PageId pageId = findPage(id.getBookId(), id.getTimestamp()).getId(); StoredMessageBatch result = doGetMessageBatch(id, pageId); logger.debug("Message batch by message ID {} got from page {}", id, pageId); return result; } - + /** * Asynchronously retrieves the batch of messages where message with given ID is stored * @param id of stored message whose batch to retrieve * @return future to obtain batch of messages * @throws CradleStorageException if given parameter is invalid */ - protected final CompletableFuture getMessageBatchAsync(StoredMessageId id) throws CradleStorageException - { + protected final CompletableFuture getMessageBatchAsync(StoredMessageId id) throws CradleStorageException { logger.debug("Getting message batch by message ID {} asynchronously", id); PageId pageId = findPage(id.getBookId(), id.getTimestamp()).getId(); CompletableFuture result = doGetMessageBatchAsync(id, pageId); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting message batch by message ID "+id+" from page "+pageId+" asynchronously", error); - else - logger.debug("Message batch by message ID {} from page {} got asynchronously", id, pageId); - }, composingService); + if (error != null) + logger.error("Error while getting message batch by message ID "+id+" from page "+pageId+" asynchronously", error); + else + logger.debug("Message batch by message ID {} from page {} got asynchronously", id, pageId); + }, composingService); return result; } - + /** * Allows enumerating stored messages filtering them by given conditions * @param filter defines conditions to filter messages by @@ -803,42 +810,40 @@ protected final CompletableFuture getMessageBatchAsync(Store * @throws IOException if data retrieval failed * @throws CradleStorageException if filter is invalid */ - public final CradleResultSet getMessages(MessageFilter filter) throws IOException, CradleStorageException - { + public final CradleResultSet getMessages(MessageFilter filter) throws IOException, CradleStorageException { logger.debug("Filtering messages by {}", filter); if (!checkFilter(filter)) return new EmptyResultSet<>(); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CradleResultSet result = doGetMessages(filter, book); logger.debug("Got result set with messages filtered by {}", filter); return result; } - + /** * Allows to asynchronously obtain result set to enumerate stored messages filtering them by given conditions * @param filter defines conditions to filter messages by * @return future to obtain result set to enumerate messages * @throws CradleStorageException if filter is invalid */ - public final CompletableFuture> getMessagesAsync(MessageFilter filter) throws CradleStorageException - { + public final CompletableFuture> getMessagesAsync(MessageFilter filter) throws CradleStorageException { logger.debug("Asynchronously getting messages filtered by {}", filter); if (!checkFilter(filter)) return CompletableFuture.completedFuture(new EmptyResultSet<>()); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CompletableFuture> result = doGetMessagesAsync(filter, book); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting messages filtered by "+filter+" asynchronously", error); - else - logger.debug("Result set with messages filtered by {} got asynchronously", filter); - }, composingService); + if (error != null) + logger.error("Error while getting messages filtered by "+filter+" asynchronously", error); + else + logger.debug("Result set with messages filtered by {} got asynchronously", filter); + }, composingService); return result; } - - + + /** * Allows enumerating stored message batches filtering them by given conditions * @param filter defines conditions to filter message batches by @@ -846,12 +851,11 @@ public final CompletableFuture> getMessagesAsync( * @throws IOException if data retrieval failed * @throws CradleStorageException if filter is invalid */ - public final CradleResultSet getMessageBatches(MessageFilter filter) throws IOException, CradleStorageException - { + public final CradleResultSet getMessageBatches(MessageFilter filter) throws IOException, CradleStorageException { logger.debug("Filtering message batches by {}", filter); if (!checkFilter(filter)) return new EmptyResultSet<>(); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CradleResultSet result = doGetMessageBatches(filter, book); logger.debug("Got result set with message batches filtered by {}", filter); @@ -867,8 +871,7 @@ public final CradleResultSet getMessageBatches(MessageFilter * @throws CradleStorageException if filter is invalid */ public final CradleResultSet getGroupedMessageBatches(GroupedMessageFilter filter) - throws CradleStorageException, IOException - { + throws CradleStorageException, IOException { logger.debug("Filtering grouped message batches by {}", filter); checkAbstractFilter(filter); @@ -877,67 +880,64 @@ public final CradleResultSet getGroupedMessageBatches logger.debug("Got result set with grouped message batches filtered by {}", filter); return result; } - - + + /** * Allows to asynchronously obtain result set to enumerate stored message batches filtering them by given conditions * @param filter defines conditions to filter message batches by * @return future to obtain result set to enumerate message batches * @throws CradleStorageException if filter is invalid */ - public final CompletableFuture> getMessageBatchesAsync(MessageFilter filter) throws CradleStorageException - { + public final CompletableFuture> getMessageBatchesAsync(MessageFilter filter) throws CradleStorageException { logger.debug("Asynchronously getting message batches filtered by {}", filter); if (!checkFilter(filter)) return CompletableFuture.completedFuture(new EmptyResultSet<>()); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CompletableFuture> result = doGetMessageBatchesAsync(filter, book); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting message batches filtered by "+filter+" asynchronously", error); - else - logger.debug("Result set with message batches filtered by {} got asynchronously", filter); - }, composingService); + if (error != null) + logger.error("Error while getting message batches filtered by "+filter+" asynchronously", error); + else + logger.debug("Result set with message batches filtered by {} got asynchronously", filter); + }, composingService); return result; } - - + + /** * Allows to asynchronously obtain result set to enumerate stored message batches filtering them by given conditions * @param filter defines conditions to filter message batches by * @return future to obtain result set to enumerate message batches * @throws CradleStorageException if filter is invalid */ - public final CompletableFuture> getGroupedMessageBatchesAsync(GroupedMessageFilter filter) throws CradleStorageException - { + public final CompletableFuture> getGroupedMessageBatchesAsync(GroupedMessageFilter filter) throws CradleStorageException { logger.debug("Asynchronously getting grouped message batches filtered by {}", filter); checkAbstractFilter(filter); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CompletableFuture> result = doGetGroupedMessageBatchesAsync(filter, book); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting message batches filtered by "+filter+" asynchronously", error); - else - logger.debug("Result set with message batches filtered by {} got asynchronously", filter); - }, composingService); + if (error != null) + logger.error("Error while getting message batches filtered by "+filter+" asynchronously", error); + else + logger.debug("Result set with message batches filtered by {} got asynchronously", filter); + }, composingService); return result; } - - + + /** - * Retrieves last stored sequence number for given session alias and direction within given page. + * Retrieves last stored sequence number for given session alias and direction within given page. * Use result of this method to continue writing messages. - * @param sessionAlias to get sequence number for + * @param sessionAlias to get sequence number for * @param direction to get sequence number for * @param bookId to get last sequence for * @return last stored sequence number for given arguments, if it is present, -1 otherwise * @throws IOException if retrieval failed * @throws CradleStorageException if given parameters are invalid */ - public final long getLastSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException - { + public final long getLastSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException { logger.debug("Getting last stored sequence number for book '{}' and session alias '{}' and direction '{}'", bookId, sessionAlias, direction.getLabel()); long result = doGetLastSequence(sessionAlias, direction, bookId); @@ -955,8 +955,7 @@ public final long getLastSequence(String sessionAlias, Direction direction, Book * @throws IOException if retrieval failed * @throws CradleStorageException if given parameters are invalid */ - public final long getFirstSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException - { + public final long getFirstSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException, CradleStorageException { logger.debug("Getting first stored sequence number for book '{}' and session alias '{}' and direction '{}'", bookId, sessionAlias, direction.getLabel()); long result = doGetFirstSequence(sessionAlias, direction, bookId); @@ -965,7 +964,6 @@ public final long getFirstSequence(String sessionAlias, Direction direction, Boo } - /** * Obtains collection of session aliases whose messages are saved in given book * @param bookId to get session aliases from @@ -973,8 +971,7 @@ public final long getFirstSequence(String sessionAlias, Direction direction, Boo * @throws IOException if data retrieval failed * @throws CradleStorageException if given book ID is invalid */ - public final Collection getSessionAliases(BookId bookId) throws IOException, CradleStorageException - { + public final Collection getSessionAliases(BookId bookId) throws IOException, CradleStorageException { logger.debug("Getting session aliases for book '{}'", bookId); getBookCache().getBook(bookId); Collection result = doGetSessionAliases(bookId); @@ -992,8 +989,8 @@ public final Collection getGroups (BookId bookId) throws IOException, Cr return result; } - - + + /** * Retrieves test event data stored under given ID * @param id of stored test event to retrieve @@ -1001,36 +998,34 @@ public final Collection getGroups (BookId bookId) throws IOException, Cr * @throws IOException if test event data retrieval failed * @throws CradleStorageException if given parameter is invalid */ - public final StoredTestEvent getTestEvent(StoredTestEventId id) throws IOException, CradleStorageException - { + public final StoredTestEvent getTestEvent(StoredTestEventId id) throws IOException, CradleStorageException { logger.debug("Getting test event {}", id); PageId pageId = findPage(id.getBookId(), id.getStartTimestamp()).getId(); StoredTestEvent result = doGetTestEvent(id, pageId); logger.debug("Test event {} got from page {}", id, pageId); return result; } - + /** * Asynchronously retrieves test event data stored under given ID * @param id of stored test event to retrieve * @return future to obtain data of stored test event * @throws CradleStorageException if given parameter is invalid */ - public final CompletableFuture getTestEventAsync(StoredTestEventId id) throws CradleStorageException - { + public final CompletableFuture getTestEventAsync(StoredTestEventId id) throws CradleStorageException { logger.debug("Getting test event {} asynchronously", id); PageId pageId = findPage(id.getBookId(), id.getStartTimestamp()).getId(); CompletableFuture result = doGetTestEventAsync(id, pageId); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting test event "+id+" from page "+pageId+" asynchronously", error); - else - logger.debug("Test event {} from page {} got asynchronously", id, pageId); - }, composingService); + if (error != null) + logger.error("Error while getting test event "+id+" from page "+pageId+" asynchronously", error); + else + logger.debug("Test event {} from page {} got asynchronously", id, pageId); + }, composingService); return result; } - - + + /** * Allows to enumerate test events, filtering them by given conditions * @param filter defines conditions to filter test events by @@ -1038,18 +1033,17 @@ public final CompletableFuture getTestEventAsync(StoredTestEven * @throws CradleStorageException if filter is invalid * @throws IOException if data retrieval failed */ - public final CradleResultSet getTestEvents(TestEventFilter filter) throws CradleStorageException, IOException - { + public final CradleResultSet getTestEvents(TestEventFilter filter) throws CradleStorageException, IOException { logger.debug("Filtering test events by {}", filter); if (!checkFilter(filter)) return new EmptyResultSet<>(); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CradleResultSet result = doGetTestEvents(filter, book); logger.debug("Got result set with test events filtered by {}", filter); return result; } - + /** * Allows to asynchronously obtain result set to enumerate test events, filtering them by given conditions * @param filter defines conditions to filter test events by @@ -1057,23 +1051,22 @@ public final CradleResultSet getTestEvents(TestEventFilter filt * @throws CradleStorageException if filter is invalid * @throws IOException if data retrieval failed */ - public final CompletableFuture> getTestEventsAsync(TestEventFilter filter) throws CradleStorageException, IOException - { + public final CompletableFuture> getTestEventsAsync(TestEventFilter filter) throws CradleStorageException, IOException { logger.debug("Asynchronously getting test events filtered by {}", filter); if (!checkFilter(filter)) return CompletableFuture.completedFuture(new EmptyResultSet<>()); - + BookInfo book = getBookCache().getBook(filter.getBookId()); CompletableFuture> result = doGetTestEventsAsync(filter, book); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while getting test events filtered by "+filter+" asynchronously", error); - else - logger.debug("Result set with test events filtered by {} got asynchronously", filter); - }, composingService); + if (error != null) + logger.error("Error while getting test events filtered by "+filter+" asynchronously", error); + else + logger.debug("Result set with test events filtered by {} got asynchronously", filter); + }, composingService); return result; } - + /** * Obtains collection of scope names whose test events are saved in given book * @param bookId to get scopes from @@ -1081,8 +1074,7 @@ public final CompletableFuture> getTestEventsAs * @throws IOException if data retrieval failed * @throws CradleStorageException if given book ID is invalid */ - public final Collection getScopes(BookId bookId) throws IOException, CradleStorageException - { + public final Collection getScopes(BookId bookId) throws IOException, CradleStorageException { logger.debug("Getting scopes for book '{}'", bookId); getBookCache().getBook(bookId); Collection result = doGetScopes(bookId); @@ -1101,10 +1093,10 @@ public final Collection getScopes(BookId bookId) throws IOException, Cra * @throws CradleStorageException if given book ID is invalid */ public CompletableFuture> getMessageCountersAsync(BookId bookId, - String sessionAlias, - Direction direction, - FrameType frameType, - Interval interval) throws CradleStorageException { + String sessionAlias, + Direction direction, + FrameType frameType, + Interval interval) throws CradleStorageException { return doGetMessageCountersAsync(bookId, sessionAlias, direction, frameType, interval); } @@ -1120,10 +1112,10 @@ public CompletableFuture> getMessageCountersAsync * @throws IOException if there is a problem with input/output */ public CradleResultSet getMessageCounters(BookId bookId, - String sessionAlias, - Direction direction, - FrameType frameType, - Interval interval) throws CradleStorageException, IOException { + String sessionAlias, + Direction direction, + FrameType frameType, + Interval interval) throws CradleStorageException, IOException { return doGetMessageCounters(bookId, sessionAlias, direction, frameType, interval); } @@ -1137,9 +1129,9 @@ public CradleResultSet getMessageCounters(BookId bookId, * @throws CradleStorageException if given book ID is invalid */ public CompletableFuture> getCountersAsync (BookId bookId, - EntityType entityType, - FrameType frameType, - Interval interval) throws CradleStorageException { + EntityType entityType, + FrameType frameType, + Interval interval) throws CradleStorageException { return doGetCountersAsync(bookId, entityType, frameType, interval); } @@ -1345,23 +1337,21 @@ private void updatePage(PageId pageId, PageInfo updatedPageInfo) throws CradleSt bookInfo.addPage(updatedPageInfo); } - public final void updateEventStatus(StoredTestEvent event, boolean success) throws IOException - { + public final void updateEventStatus(StoredTestEvent event, boolean success) throws IOException { logger.debug("Updating status of event {}", event.getId()); doUpdateEventStatus(event, success); logger.debug("Status of event {} has been updated", event.getId()); } - public final CompletableFuture updateEventStatusAsync(StoredTestEvent event, boolean success) - { + public final CompletableFuture updateEventStatusAsync(StoredTestEvent event, boolean success) { logger.debug("Asynchronously updating status of event {}", event.getId()); CompletableFuture result = doUpdateEventStatusAsync(event, success); result.whenCompleteAsync((r, error) -> { - if (error != null) - logger.error("Error while asynchronously updating status of event "+event.getId()); - else - logger.debug("Status of event {} updated asynchronously", event.getId()); - }, composingService); + if (error != null) + logger.error("Error while asynchronously updating status of event "+event.getId()); + else + logger.debug("Status of event {} updated asynchronously", event.getId()); + }, composingService); return result; } @@ -1382,36 +1372,34 @@ private Instant checkCollisionAndGetPageEnd(BookInfo book, PageToAdd page, Insta return pageInBookAfterStart == null ? defaultPageEnd : pageInBookAfterStart.getStarted(); } - - private List checkPages(List pages, BookInfo book) throws CradleStorageException - { + + private List checkPages(List pages, BookInfo book) throws CradleStorageException { PageInfo lastPage = book.getLastPage(); if (lastPage != null) //If book has any pages, i.e. may have some data { + // TODO: shouldn't we use 'lastPage.getStart()' to validate that new page is not in the past? Instant now = Instant.now(), firstStart = pages.get(0).getStart(); if (!firstStart.isAfter(now)) throw new CradleStorageException("Timestamp of new page start must be after current timestamp ("+now+")"); } - + Set names = new HashSet<>(); PageToAdd prevPage = null; BookId bookId = book.getId(); List result = new ArrayList<>(pages.size()); - for (PageToAdd page : pages) - { + for (PageToAdd page : pages) { BookPagesNamesChecker.validatePageName(page.getName()); String name = page.getName(); if (names.contains(name)) throw new CradleStorageException("Duplicated page name: '"+page.getName()+"'"); names.add(name); - + if (book.getPage(new PageId(bookId, name)) != null) throw new CradleStorageException("Page '"+name+"' is already present in book '"+bookId+"'"); - - if (prevPage != null) - { + + if (prevPage != null) { if (!page.getStart().isAfter(prevPage.getStart())) { throw new CradleStorageException("Unordered pages: page '"+name+"' should start after page '"+prevPage.getName()+"'"); } @@ -1433,52 +1421,48 @@ private List checkPages(List pages, BookInfo book) throws C return result; } - - private boolean checkFilter(MessageFilter filter) throws CradleStorageException - { + + private boolean checkFilter(MessageFilter filter) throws CradleStorageException { checkAbstractFilter(filter); - + //TODO: add more checks return true; } - private void checkAbstractFilter(AbstractFilter filter) throws CradleStorageException - { + private void checkAbstractFilter(AbstractFilter filter) throws CradleStorageException { BookInfo book = getBookCache().getBook(filter.getBookId()); if (filter.getPageId() != null) checkPage(filter.getPageId(), book.getId()); } - - private boolean checkFilter(TestEventFilter filter) throws CradleStorageException - { + + private boolean checkFilter(TestEventFilter filter) throws CradleStorageException { BookInfo book = getBookCache().getBook(filter.getBookId()); checkAbstractFilter(filter); - + if (filter.getParentId() != null && !book.getId().equals(filter.getParentId().getBookId())) throw new CradleStorageException("Requested book ("+book.getId()+") doesn't match book of requested parent ("+filter.getParentId()+")"); - + Instant timeFrom = filter.getStartTimestampFrom() != null ? filter.getStartTimestampFrom().getValue() : null, timeTo = filter.getStartTimestampTo() != null ? filter.getStartTimestampTo().getValue() : null; - if (timeFrom != null && timeTo != null + if (timeFrom != null && timeTo != null && timeFrom.isAfter(timeTo)) throw new CradleStorageException("Left bound for start timestamp ("+timeFrom+") " + "is after the right bound ("+timeTo+")"); - + if (timeTo != null && timeTo.isBefore(book.getCreated())) return false; - - if (filter.getPageId() != null) - { + + if (filter.getPageId() != null) { PageInfo page = book.getPage(filter.getPageId()); Instant pageStarted = page.getStarted(), pageEnded = page.getEnded(); - + if (timeFrom != null && pageEnded != null && timeFrom.isAfter(pageEnded)) return false; if (timeTo != null && timeTo.isBefore(pageStarted)) return false; } - + return true; } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilter.java b/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilter.java index f78f893d3..c3fd854ba 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilter.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilter.java @@ -36,19 +36,16 @@ public abstract class AbstractFilter private int limit; private Order order = Order.DIRECT; - protected AbstractFilter(BookId bookId) - { + protected AbstractFilter(BookId bookId) { this.bookId = bookId; } - - protected AbstractFilter(BookId bookId, PageId pageId) - { + + protected AbstractFilter(BookId bookId, PageId pageId) { this(bookId); this.pageId = pageId; } - protected AbstractFilter(AbstractFilter copyFrom) - { + protected AbstractFilter(AbstractFilter copyFrom) { this.bookId = copyFrom.getBookId(); this.pageId = copyFrom.getPageId(); this.from = copyFrom.getFrom(); @@ -58,78 +55,74 @@ protected AbstractFilter(AbstractFilter copyFrom) this.fetchParameters = copyFrom.getFetchParameters(); } - public BookId getBookId() - { + public BookId getBookId() { return bookId; } - public PageId getPageId() - { + public PageId getPageId() { return pageId; } - public void setPageId(PageId pageId) - { + public void setPageId(PageId pageId) { this.pageId = pageId; } - protected FilterForGreater getFrom() - { + protected FilterForGreater getFrom() { return from; } - protected void setFrom(FilterForGreater from) - { + protected void setFrom(FilterForGreater from) { this.from = from; } - protected FilterForLess getTo() - { + protected FilterForLess getTo() { return to; } - protected void setTo(FilterForLess to) - { + protected void setTo(FilterForLess to) { this.to = to; } - public int getLimit() - { + public int getLimit() { return limit; } - public void setLimit(int limit) - { + public void setLimit(int limit) { this.limit = limit; } - public Order getOrder() - { + public Order getOrder() { return order; } - public void setOrder(Order order) - { + public void setOrder(Order order) { this.order = order == null ? Order.DIRECT : order; } - public FetchParameters getFetchParameters() - { + public FetchParameters getFetchParameters() { return fetchParameters; } - public void setFetchParameters(FetchParameters fetchParameters) - { + public void setFetchParameters(FetchParameters fetchParameters) { this.fetchParameters = fetchParameters; } - protected void validate() throws CradleStorageException - { - if (bookId == null) + protected void validate() throws CradleStorageException { + if (bookId == null) { throw new CradleStorageException("bookId is mandatory"); - if (pageId != null && !pageId.getBookId().equals(bookId)) - throw new CradleStorageException("pageId must be from book '"+bookId+"'"); + } + if (pageId != null && !pageId.getBookId().equals(bookId)) { + throw new CradleStorageException("pageId must be from book '" + bookId + "'"); + } + if (from != null && to != null) { + Instant fromValue = from.getValue(); + Instant toValue = to.getValue(); + if (fromValue.isAfter(toValue)) { + throw new CradleStorageException(String.format("'from' (%s) must be less or equal to 'to' (%s)", + fromValue, toValue)); + } + } } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilterBuilder.java b/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilterBuilder.java index 3a9384733..330f9ca5a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilterBuilder.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/filters/AbstractFilterBuilder.java @@ -34,73 +34,61 @@ public abstract class AbstractFilterBuilder private int limit; private Order order = Order.DIRECT; - public BookId getBookId() - { + public BookId getBookId() { return bookId; } - public PageId getPageId() - { + public PageId getPageId() { return pageId; } - public FetchParameters getFetchParameters() - { + public FetchParameters getFetchParameters() { return fetchParameters; } - public FilterForGreater getTimestampFrom() - { + public FilterForGreater getTimestampFrom() { return timestampFrom; } - public FilterForLess getTimestampTo() - { + public FilterForLess getTimestampTo() { return timestampTo; } - public int getLimit() - { + public int getLimit() { return limit; } - public Order getOrder() - { + public Order getOrder() { return order; } @SuppressWarnings("unchecked") - public T bookId(BookId bookId) - { + public T bookId(BookId bookId) { this.bookId = bookId; return (T) this; } @SuppressWarnings("unchecked") - public T pageId(PageId pageId) - { + public T pageId(PageId pageId) { this.pageId = pageId; return (T) this; } @SuppressWarnings("unchecked") - public T fetchParameters(FetchParameters fetchParameters) - { + public T fetchParameters(FetchParameters fetchParameters) { this.fetchParameters = fetchParameters; return (T) this; } @SuppressWarnings("unchecked") - public FilterForGreaterBuilder timestampFrom() - { + public FilterForGreaterBuilder timestampFrom() { FilterForGreater f = new FilterForGreater<>(); timestampFrom = f; return new FilterForGreaterBuilder<>(f, (T) this); } @SuppressWarnings("unchecked") - public FilterForLessBuilder timestampTo() - { + public FilterForLessBuilder timestampTo() { FilterForLess f = new FilterForLess<>(); timestampTo = f; return new FilterForLessBuilder<>(f, (T) this); @@ -112,21 +100,18 @@ public FilterForLessBuilder timestampTo() * @return the same builder instance to continue building chain */ @SuppressWarnings("unchecked") - public T limit(int limit) - { + public T limit(int limit) { this.limit = limit; return (T) this; } @SuppressWarnings("unchecked") - public T order(Order order) - { + public T order(Order order) { this.order = order; return (T) this; } - protected void reset() - { + protected void reset() { bookId = null; pageId = null; fetchParameters = null; @@ -136,15 +121,14 @@ protected void reset() order = Order.DIRECT; } - public R build() throws CradleStorageException - { + public R build() throws CradleStorageException { R instance = createFilterInstance(); instance.setFetchParameters(fetchParameters); instance.setFrom(timestampFrom); instance.setTo(timestampTo); instance.setLimit(limit); instance.setOrder(order); - + instance.validate(); return instance; } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageFilter.java b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageFilter.java index 26becef3a..b4348805a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageFilter.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/messages/GroupedMessageFilter.java @@ -21,6 +21,8 @@ import com.exactpro.cradle.filters.AbstractFilter; import com.exactpro.cradle.filters.FilterForGreater; import com.exactpro.cradle.filters.FilterForLess; +import com.exactpro.cradle.utils.CradleStorageException; +import org.apache.commons.lang3.StringUtils; import java.time.Instant; @@ -28,61 +30,59 @@ public class GroupedMessageFilter extends AbstractFilter { private final String groupName; - public GroupedMessageFilter(BookId bookId, String groupName) - { + public GroupedMessageFilter(BookId bookId, String groupName) { super(bookId); this.groupName = groupName; } - public GroupedMessageFilter(BookId bookId, PageId pageId, String groupName) - { + public GroupedMessageFilter(BookId bookId, PageId pageId, String groupName) { super(bookId, pageId); this.groupName = groupName; } - public GroupedMessageFilter(GroupedMessageFilter copyFrom) - { + public GroupedMessageFilter(GroupedMessageFilter copyFrom) { super(copyFrom); this.groupName = copyFrom.groupName; } - - public static GroupedMessageFilterBuilder builder() - { + + public static GroupedMessageFilterBuilder builder() { return new GroupedMessageFilterBuilder(); } @Override - public void setFrom(FilterForGreater from) - { + public void setFrom(FilterForGreater from) { super.setFrom(from); } @Override - public void setTo(FilterForLess to) - { + public void setTo(FilterForLess to) { super.setTo(to); } - public String getGroupName() - { + public String getGroupName() { return groupName; } @Override - public FilterForGreater getFrom() - { + public FilterForGreater getFrom() { return super.getFrom(); } @Override - public FilterForLess getTo() - { + public FilterForLess getTo() { return super.getTo(); } @Override - public String toString() - { + protected void validate() throws CradleStorageException { + super.validate(); + if (StringUtils.isBlank(groupName)) { + throw new CradleStorageException("groupName is blank"); + } + } + + @Override + public String toString() { StringBuilder sb = new StringBuilder("["); if (getBookId() != null) sb.append("bookId=").append(getBookId()).append(TO_STRING_DELIMITER); diff --git a/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageFilterBuilderTest.java b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageFilterBuilderTest.java new file mode 100644 index 000000000..2685285f6 --- /dev/null +++ b/cradle-core/src/test/java/com/exactpro/cradle/messages/GroupedMessageFilterBuilderTest.java @@ -0,0 +1,109 @@ +/* + * Copyright 2023 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.cradle.messages; + +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.utils.CradleStorageException; +import org.testng.annotations.DataProvider; +import org.testng.annotations.Test; + +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNull; + +public class GroupedMessageFilterBuilderTest { + + private static final BookId BOOK_ID = new BookId("test"); + + @Test( + expectedExceptions = CradleStorageException.class, + expectedExceptionsMessageRegExp = "groupName is blank" + ) + public void testCheckGroupName() throws CradleStorageException { + GroupedMessageFilter.builder() +// .groupName() // skip group name + .bookId(BOOK_ID) + .timestampFrom().isGreaterThanOrEqualTo(Instant.MIN) + .timestampTo().isLessThan(Instant.MAX) + .build(); + } + + @Test + public void testCreatesWithOnlyFromTimestamp() throws CradleStorageException { + Instant from = Instant.now(); + var filter = GroupedMessageFilter.builder() + .bookId(BOOK_ID) + .groupName("test") + .timestampFrom().isGreaterThanOrEqualTo(from) + .build(); + + assertEquals(filter.getFrom().getValue(), from, "unexpected 'from' value"); + assertNull(filter.getTo(), "unexpected 'to' value"); + } + + @Test + public void testCreatesWithOnlyToTimestamp() throws CradleStorageException { + Instant to = Instant.now(); + var filter = GroupedMessageFilter.builder() + .bookId(BOOK_ID) + .groupName("test") + .timestampTo().isLessThan(to) + .build(); + + assertEquals(filter.getTo().getValue(), to, "unexpected 'to' value"); + assertNull(filter.getFrom(), "unexpected 'from' value"); + } + + @DataProvider(name = "timestamps") + public static Object[][] timestamps() { + return new Object[][]{ + { Instant.now(), Instant.now().plus(1, ChronoUnit.DAYS) }, + { Instant.ofEpochSecond(1000), Instant.ofEpochSecond(1000) } + }; + } + + @Test( + dataProvider = "timestamps" + ) + public void testCreatesWithBothTimestamp(Instant from, Instant to) throws CradleStorageException { + var filter = GroupedMessageFilter.builder() + .bookId(BOOK_ID) + .groupName("test") + .timestampFrom().isGreaterThanOrEqualTo(from) + .timestampTo().isLessThan(to) + .build(); + + assertEquals(filter.getFrom().getValue(), from, "unexpected 'from' value"); + assertEquals(filter.getTo().getValue(), to, "unexpected 'to' value"); + } + + @Test( + expectedExceptions = CradleStorageException.class, + expectedExceptionsMessageRegExp = "'from' \\(.*\\) must be less or equal to 'to' \\(.*\\)" + ) + public void testCheckTimestamps() throws CradleStorageException { + var from = Instant.now(); + var to = from.minus(1, ChronoUnit.DAYS); + GroupedMessageFilter.builder() + .bookId(BOOK_ID) + .groupName("test") + .timestampFrom().isGreaterThanOrEqualTo(from) + .timestampTo().isLessThan(to) + .build(); + } +} \ No newline at end of file From 5f78882468398c63dc550934614caf93f77a2efa Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Tue, 6 Jun 2023 19:57:28 +0400 Subject: [PATCH 21/31] Added metrics * cradle_test_event_batches_read_total * cradle_test_event_batches_stored_total * cradle_message_batch_read_total * cradle_message_batch_stored_total --- .../cassandra/workers/EventsWorker.java | 421 +++++---- .../cassandra/workers/MessagesWorker.java | 857 ++++++++++-------- .../cassandra/workers/MetricHolder.java | 46 + .../cradle/cassandra/workers/Worker.java | 70 +- 4 files changed, 789 insertions(+), 605 deletions(-) create mode 100644 cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MetricHolder.java diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java index de991e2d4..873ad0dba 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/EventsWorker.java @@ -26,7 +26,12 @@ import com.exactpro.cradle.cassandra.dao.CassandraOperators; import com.exactpro.cradle.cassandra.dao.cache.CachedPageScope; import com.exactpro.cradle.cassandra.dao.cache.CachedScope; -import com.exactpro.cradle.cassandra.dao.testevents.*; +import com.exactpro.cradle.cassandra.dao.testevents.PageScopeEntity; +import com.exactpro.cradle.cassandra.dao.testevents.ScopeEntity; +import com.exactpro.cradle.cassandra.dao.testevents.TestEventEntity; +import com.exactpro.cradle.cassandra.dao.testevents.TestEventEntityUtils; +import com.exactpro.cradle.cassandra.dao.testevents.TestEventIteratorProvider; +import com.exactpro.cradle.cassandra.dao.testevents.TestEventOperator; import com.exactpro.cradle.cassandra.dao.testevents.converters.TestEventEntityConverter; import com.exactpro.cradle.cassandra.resultset.CassandraCradleResultSet; import com.exactpro.cradle.resultset.CradleResultSet; @@ -43,193 +48,247 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import java.io.IOException; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; import java.util.List; +import java.util.Objects; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.zip.DataFormatException; -public class EventsWorker extends Worker -{ - private static final Logger logger = LoggerFactory.getLogger(EventsWorker.class); - - private static final Counter EVENTS_READ_METRIC = Counter.build().name("cradle_test_events_read_total") - .help("Fetched test events").labelNames(BOOK_ID, SCOPE).register(); - private static final Counter EVENTS_STORE_METRIC = Counter.build().name("cradle_test_events_stored_total") - .help("Stored test events").labelNames(BOOK_ID, SCOPE).register(); - private static final Counter EVENTS_STORE_UNCOMPRESSED_BYTES = Counter.build().name("cradle_test_events_stored_uncompressed_bytes_total") - .help("Stored uncompressed event bytes").labelNames(BOOK_ID, SCOPE).register(); - private static final Counter EVENTS_STORE_COMPRESSED_BYTES = Counter.build().name("cradle_test_events_stored_compressed_bytes_total") - .help("Stored compressed event bytes").labelNames(BOOK_ID, SCOPE).register(); - - private final EntityStatisticsCollector entityStatisticsCollector; - private final EventBatchDurationWorker durationWorker; - - public EventsWorker(WorkerSupplies workerSupplies, EntityStatisticsCollector entityStatisticsCollector, EventBatchDurationWorker durationWorker) - { - super(workerSupplies); - this.entityStatisticsCollector = entityStatisticsCollector; - this.durationWorker = durationWorker; - } - - public static StoredTestEvent mapTestEventEntity(PageId pageId, TestEventEntity entity) - { - try - { - StoredTestEvent testEvent = TestEventEntityUtils.toStoredTestEvent(entity, pageId); - updateEventReadMetrics(testEvent); - return testEvent; - } - catch (DataFormatException | CradleStorageException | CradleIdException | IOException | CompressException e) - { - throw new CompletionException("Error while converting test event entity into Cradle test event", e); - } - } - - private static void updateEventReadMetrics(StoredTestEvent testEvent) - { - EVENTS_READ_METRIC.labels(testEvent.getId().getBookId().getName(), testEvent.getScope()) - .inc(testEvent.isSingle() ? 1 : testEvent.asBatch().getTestEventsCount()); - } - - private static void updateEventWriteMetrics(TestEventEntity entity, BookId bookId) { - EVENTS_STORE_METRIC.labels(bookId.getName(), entity.getScope()) - .inc(entity.isEventBatch() ? entity.getEventCount() : 1); - - EVENTS_STORE_UNCOMPRESSED_BYTES.labels(bookId.getName(), entity.getScope()) - .inc(entity.getUncompressedContentSize()); - EVENTS_STORE_COMPRESSED_BYTES.labels(bookId.getName(), entity.getScope()) - .inc(entity.getContentSize()); - } - - public CompletableFuture storeEvent(TestEventToStore event, PageId pageId) - { - TestEventOperator op = getOperators().getTestEventOperator(); - BookStatisticsRecordsCaches.EntityKey key = new BookStatisticsRecordsCaches.EntityKey(pageId.getName(), EntityType.EVENT); - - return CompletableFuture.supplyAsync(() -> { - try { - return TestEventEntityUtils.toSerializedEntity(event, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); - } catch (Exception e) { - throw new CompletionException(e); - } - }, composingService).thenCompose(serializedEntity -> { - TestEventEntity entity = serializedEntity.getEntity(); - List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); - - return op.write(entity, writeAttrs) - .thenAcceptAsync(result -> { - try { - Instant firstTimestamp = meta.get(0).getTimestamp(); - Instant lastStartTimestamp = firstTimestamp; - for (SerializedEntityMetadata el : meta) { - if (el.getTimestamp() != null) { - if (firstTimestamp.isAfter(el.getTimestamp())) { - firstTimestamp = el.getTimestamp(); - } - if (lastStartTimestamp.isBefore(el.getTimestamp())) { - lastStartTimestamp = el.getTimestamp(); - } - } - } - durationWorker.updateMaxDuration(pageId, entity.getScope(), - Duration.between(firstTimestamp, lastStartTimestamp).toMillis(), - writeAttrs); - } catch (CradleStorageException e) { - logger.error("Exception while updating max duration {}", e.getMessage()); - } - - entityStatisticsCollector.updateEntityBatchStatistics(pageId.getBookId(), key, meta); - updateEventWriteMetrics(entity, pageId.getBookId()); - }, composingService); - }); - } - - public CompletableFuture storeScope(TestEventToStore event) - { - String bookName = event.getBookId().getName(); - CassandraOperators operators = getOperators(); - if (!operators.getScopesCache().store(new CachedScope(bookName, event.getScope()))) - { - logger.debug("Skipped writing scope of event '{}'", event.getId()); - return CompletableFuture.completedFuture(null); - } - - logger.debug("Writing scope of event '{}'", event.getId()); - return operators.getScopeOperator() - .write(new ScopeEntity(bookName, event.getScope()), writeAttrs); - } - - public CompletableFuture storePageScope(TestEventToStore event, PageId pageId) - { - CassandraOperators operators = getOperators(); - if (!operators.getPageScopesCache().store(new CachedPageScope(pageId.toString(), event.getScope()))) - { - logger.debug("Skipped writing scope partition of event '{}'", event.getId()); - return CompletableFuture.completedFuture(null); - } - - logger.debug("Writing scope partition of event '{}'", event.getId()); - return operators.getPageScopesOperator() - .write(new PageScopeEntity(pageId.getBookId().getName(), pageId.getName(), event.getScope()), writeAttrs); - } - - public CompletableFuture getTestEvent(StoredTestEventId id, PageId pageId) - { - LocalDateTime ldt = TimeUtils.toLocalTimestamp(id.getStartTimestamp()); - CassandraOperators operators = getOperators(); - TestEventEntityConverter converter = operators.getTestEventEntityConverter(); - return selectQueryExecutor.executeSingleRowResultQuery( - () -> operators.getTestEventOperator().get(pageId.getBookId().getName(), pageId.getName(), id.getScope(), - ldt.toLocalDate(), ldt.toLocalTime(), id.getId(), readAttrs), converter::getEntity, - String.format("get test event by id '%s'", id)) - .thenApplyAsync(entity -> { - if (entity == null) - return null; - - try - { - return TestEventEntityUtils.toStoredTestEvent(entity, pageId); - } - catch (Exception e) - { - throw new CompletionException("Error while converting data of event "+id+" into test event", e); - } - }, composingService); - } - - public CompletableFuture> getTestEvents(TestEventFilter filter, BookInfo book) - { - Instant startTimestamp; - - if (filter.getStartTimestampFrom() == null) { - if (filter.getPageId() == null) { - startTimestamp = book.getFirstPage().getStarted(); - } else { - startTimestamp = book.getPage(filter.getPageId()).getStarted(); - } - } else { - startTimestamp = filter.getStartTimestampFrom().getValue(); - } - - TestEventIteratorProvider provider = new TestEventIteratorProvider("get test events filtered by "+filter, - filter, getOperators(), book, composingService, selectQueryExecutor, - durationWorker, - composeReadAttrs(filter.getFetchParameters()), - startTimestamp); - return provider.nextIterator() - .thenApply(r -> new CassandraCradleResultSet<>(r, provider)); - } - - public CompletableFuture updateStatus(StoredTestEvent event, boolean success) - { - StoredTestEventId id = event.getId(); - LocalDateTime ldt = TimeUtils.toLocalTimestamp(event.getStartTimestamp()); - PageId pageId = event.getPageId(); - return getOperators().getTestEventOperator().updateStatus(pageId.getBookId().getName(), pageId.getName(), id.getScope(), - ldt.toLocalDate(), ldt.toLocalTime(), id.getId(), success, writeAttrs); - } +import static java.util.Objects.requireNonNull; + +public class EventsWorker extends Worker { + private static final Logger logger = LoggerFactory.getLogger(EventsWorker.class); + + private static final MetricHolder EVENTS_READ_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_test_events_read_total") + .help("Fetched test events") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder EVENT_BATCHES_READ_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_test_event_batches_read_total") + .help("Fetched test event batches") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder EVENTS_STORE_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_test_events_stored_total") + .help("Stored test events") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder EVENT_BATCHES_STORE_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_test_event_batches_stored_total") + .help("Stored test event batches") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder EVENTS_STORE_UNCOMPRESSED_BYTES = new MetricHolder<>( + Counter.build() + .name("cradle_test_events_stored_uncompressed_bytes_total") + .help("Stored uncompressed event bytes") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder EVENTS_STORE_COMPRESSED_BYTES = new MetricHolder<>( + Counter.build() + .name("cradle_test_events_stored_compressed_bytes_total") + .help("Stored compressed event bytes") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + + private final EntityStatisticsCollector entityStatisticsCollector; + private final EventBatchDurationWorker durationWorker; + + public EventsWorker(WorkerSupplies workerSupplies, EntityStatisticsCollector entityStatisticsCollector, EventBatchDurationWorker durationWorker) { + super(workerSupplies); + this.entityStatisticsCollector = entityStatisticsCollector; + this.durationWorker = durationWorker; + } + + public static StoredTestEvent mapTestEventEntity(PageId pageId, TestEventEntity entity) { + try { + StoredTestEvent testEvent = TestEventEntityUtils.toStoredTestEvent(entity, pageId); + updateEventReadMetrics(testEvent); + return testEvent; + } catch (DataFormatException | CradleStorageException | CradleIdException | IOException | CompressException e) { + throw new CompletionException("Error while converting test event entity into Cradle test event", e); + } + } + + private static void updateEventReadMetrics(StoredTestEvent testEvent) { + StreamLabel key = new StreamLabel(testEvent.getId().getBookId().getName(), testEvent.getScope()); + EVENTS_READ_METRIC.inc(key, testEvent.isSingle() ? 1 : testEvent.asBatch().getTestEventsCount()); + EVENT_BATCHES_READ_METRIC.inc(key); + } + + private static void updateEventWriteMetrics(TestEventEntity entity, BookId bookId) { + StreamLabel key = new StreamLabel(bookId.getName(), entity.getScope()); + EVENTS_STORE_METRIC.inc(key, entity.isEventBatch() ? entity.getEventCount() : 1); + EVENT_BATCHES_STORE_METRIC.inc(key); + EVENTS_STORE_UNCOMPRESSED_BYTES.inc(key, entity.getUncompressedContentSize()); + EVENTS_STORE_COMPRESSED_BYTES.inc(key, entity.getContentSize()); + } + + public CompletableFuture storeEvent(TestEventToStore event, PageId pageId) { + TestEventOperator op = getOperators().getTestEventOperator(); + BookStatisticsRecordsCaches.EntityKey key = new BookStatisticsRecordsCaches.EntityKey(pageId.getName(), EntityType.EVENT); + + return CompletableFuture.supplyAsync(() -> { + try { + return TestEventEntityUtils.toSerializedEntity(event, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); + } catch (Exception e) { + throw new CompletionException(e); + } + }, composingService).thenCompose(serializedEntity -> { + TestEventEntity entity = serializedEntity.getEntity(); + List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); + + return op.write(entity, writeAttrs) + .thenAcceptAsync(result -> { + try { + Instant firstTimestamp = meta.get(0).getTimestamp(); + Instant lastStartTimestamp = firstTimestamp; + for (SerializedEntityMetadata el : meta) { + if (el.getTimestamp() != null) { + if (firstTimestamp.isAfter(el.getTimestamp())) { + firstTimestamp = el.getTimestamp(); + } + if (lastStartTimestamp.isBefore(el.getTimestamp())) { + lastStartTimestamp = el.getTimestamp(); + } + } + } + durationWorker.updateMaxDuration(pageId, entity.getScope(), + Duration.between(firstTimestamp, lastStartTimestamp).toMillis(), + writeAttrs); + } catch (CradleStorageException e) { + logger.error("Exception while updating max duration {}", e.getMessage()); + } + + entityStatisticsCollector.updateEntityBatchStatistics(pageId.getBookId(), key, meta); + updateEventWriteMetrics(entity, pageId.getBookId()); + }, composingService); + }); + } + + public CompletableFuture storeScope(TestEventToStore event) { + String bookName = event.getBookId().getName(); + CassandraOperators operators = getOperators(); + if (!operators.getScopesCache().store(new CachedScope(bookName, event.getScope()))) { + logger.debug("Skipped writing scope of event '{}'", event.getId()); + return CompletableFuture.completedFuture(null); + } + + logger.debug("Writing scope of event '{}'", event.getId()); + return operators.getScopeOperator() + .write(new ScopeEntity(bookName, event.getScope()), writeAttrs); + } + + public CompletableFuture storePageScope(TestEventToStore event, PageId pageId) { + CassandraOperators operators = getOperators(); + if (!operators.getPageScopesCache().store(new CachedPageScope(pageId.toString(), event.getScope()))) { + logger.debug("Skipped writing scope partition of event '{}'", event.getId()); + return CompletableFuture.completedFuture(null); + } + + logger.debug("Writing scope partition of event '{}'", event.getId()); + return operators.getPageScopesOperator() + .write(new PageScopeEntity(pageId.getBookId().getName(), pageId.getName(), event.getScope()), writeAttrs); + } + + public CompletableFuture getTestEvent(StoredTestEventId id, PageId pageId) { + LocalDateTime ldt = TimeUtils.toLocalTimestamp(id.getStartTimestamp()); + CassandraOperators operators = getOperators(); + TestEventEntityConverter converter = operators.getTestEventEntityConverter(); + return selectQueryExecutor.executeSingleRowResultQuery( + () -> operators.getTestEventOperator().get(pageId.getBookId().getName(), pageId.getName(), id.getScope(), + ldt.toLocalDate(), ldt.toLocalTime(), id.getId(), readAttrs), converter::getEntity, + String.format("get test event by id '%s'", id)) + .thenApplyAsync(entity -> { + if (entity == null) + return null; + + try { + return TestEventEntityUtils.toStoredTestEvent(entity, pageId); + } catch (Exception e) { + throw new CompletionException("Error while converting data of event " + id + " into test event", e); + } + }, composingService); + } + + public CompletableFuture> getTestEvents(TestEventFilter filter, BookInfo book) { + Instant startTimestamp; + + if (filter.getStartTimestampFrom() == null) { + if (filter.getPageId() == null) { + startTimestamp = book.getFirstPage().getStarted(); + } else { + startTimestamp = book.getPage(filter.getPageId()).getStarted(); + } + } else { + startTimestamp = filter.getStartTimestampFrom().getValue(); + } + + TestEventIteratorProvider provider = new TestEventIteratorProvider("get test events filtered by " + filter, + filter, getOperators(), book, composingService, selectQueryExecutor, + durationWorker, + composeReadAttrs(filter.getFetchParameters()), + startTimestamp); + return provider.nextIterator() + .thenApply(r -> new CassandraCradleResultSet<>(r, provider)); + } + + public CompletableFuture updateStatus(StoredTestEvent event, boolean success) { + StoredTestEventId id = event.getId(); + LocalDateTime ldt = TimeUtils.toLocalTimestamp(event.getStartTimestamp()); + PageId pageId = event.getPageId(); + return getOperators().getTestEventOperator().updateStatus(pageId.getBookId().getName(), pageId.getName(), id.getScope(), + ldt.toLocalDate(), ldt.toLocalTime(), id.getId(), success, writeAttrs); + } + + private static class StreamLabel implements MetricHolder.LabelHolder { + private static final String[] LABEL_NAMES = new String[]{BOOK_ID, SCOPE}; + private final String bookId; + private final String scope; + private final int hash; + + private StreamLabel( + @Nonnull String bookId, + @Nonnull String scope + ) { + this.bookId = requireNonNull(bookId, "'bookId' can't be null"); + this.scope = requireNonNull(scope, "'scope' can't be null"); + this.hash = Objects.hash(bookId, scope); + } + + @Override + public String[] getLabels() { + return new String[]{bookId, scope}; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + StreamLabel that = (StreamLabel) o; + return Objects.equals(bookId, that.bookId) + && Objects.equals(scope, that.scope); + } + + @Override + public int hashCode() { + return hash; + } + } } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java index a74d09552..9cf8ac875 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java @@ -18,18 +18,40 @@ import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; import com.datastax.oss.driver.api.core.cql.Row; -import com.exactpro.cradle.*; +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.BookInfo; +import com.exactpro.cradle.Direction; +import com.exactpro.cradle.PageId; +import com.exactpro.cradle.PageInfo; +import com.exactpro.cradle.SessionRecordType; import com.exactpro.cradle.cassandra.counters.MessageStatisticsCollector; import com.exactpro.cradle.cassandra.counters.SessionStatisticsCollector; import com.exactpro.cradle.cassandra.dao.CassandraOperators; import com.exactpro.cradle.cassandra.dao.cache.CachedPageSession; import com.exactpro.cradle.cassandra.dao.cache.CachedSession; -import com.exactpro.cradle.cassandra.dao.messages.*; +import com.exactpro.cradle.cassandra.dao.messages.GroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchesIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.MessagesIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.PageGroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.PageSessionEntity; +import com.exactpro.cradle.cassandra.dao.messages.SessionEntity; import com.exactpro.cradle.cassandra.dao.messages.converters.MessageBatchEntityConverter; import com.exactpro.cradle.cassandra.resultset.CassandraCradleResultSet; import com.exactpro.cradle.cassandra.utils.GroupedMessageEntityUtils; import com.exactpro.cradle.cassandra.utils.MessageBatchEntityUtils; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.GroupedMessageFilter; +import com.exactpro.cradle.messages.MessageBatchToStore; +import com.exactpro.cradle.messages.MessageFilter; +import com.exactpro.cradle.messages.StoredGroupedMessageBatch; +import com.exactpro.cradle.messages.StoredMessage; +import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.resultset.CradleResultSet; import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.utils.CompressException; @@ -39,11 +61,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.annotation.Nonnull; import java.io.IOException; import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -52,391 +76,446 @@ import java.util.zip.DataFormatException; import static com.exactpro.cradle.CradleStorage.EMPTY_MESSAGE_INDEX; -import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.*; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_FIRST_MESSAGE_TIME; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_LAST_SEQUENCE; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_SEQUENCE; import static java.lang.String.format; - -public class MessagesWorker extends Worker -{ - private static final Logger logger = LoggerFactory.getLogger(MessagesWorker.class); - - private static final Counter MESSAGE_READ_METRIC = Counter.build().name("cradle_message_read_total") - .help("Fetched messages").labelNames(BOOK_ID, SESSION_ALIAS, DIRECTION).register(); - private static final Counter MESSAGE_STORE_METRIC = Counter.build().name("cradle_message_stored_total") - .help("Stored messages").labelNames(BOOK_ID, SESSION_ALIAS, DIRECTION).register(); - private static final Counter MESSAGE_STORE_UNCOMPRESSED_BYTES = Counter.build().name("cradle_message_stored_uncompressed_bytes_total") - .help("Stored uncompressed message bytes").labelNames(BOOK_ID, SESSION_ALIAS).register(); - private static final Counter MESSAGE_STORE_COMPRESSED_BYTES = Counter.build().name("cradle_message_stored_compressed_bytes_total") - .help("Stored compressed message bytes").labelNames(BOOK_ID, SESSION_ALIAS).register(); - - private final MessageStatisticsCollector messageStatisticsCollector; - private final SessionStatisticsCollector sessionStatisticsCollector; - public MessagesWorker(WorkerSupplies workerSupplies - , MessageStatisticsCollector messageStatisticsCollector - , SessionStatisticsCollector sessionStatisticsCollector) - { - super(workerSupplies); - this.messageStatisticsCollector = messageStatisticsCollector; - this.sessionStatisticsCollector = sessionStatisticsCollector; - } - - public static StoredMessageBatch mapMessageBatchEntity(PageId pageId, MessageBatchEntity entity) - { - try - { - StoredMessageBatch batch = MessageBatchEntityUtils.toStoredMessageBatch(entity, pageId); - updateMessageReadMetrics(batch); - return batch; - } - catch (DataFormatException | IOException | CompressException e) - { - throw new CompletionException("Error while converting message batch entity into stored message batch", e); - } - } - - public static StoredGroupedMessageBatch mapGroupedMessageBatchEntity(PageId pageId, GroupedMessageBatchEntity entity) - { - try - { - StoredGroupedMessageBatch batch = GroupedMessageEntityUtils.toStoredGroupedMessageBatch(entity, pageId); - updateMessageReadMetrics(pageId.getBookId(), batch); - return batch; - } - catch (DataFormatException | IOException | CompressException e) - { - throw new CompletionException("Error while converting message batch entity into stored message batch", e); - } - } - - private static void updateMessageReadMetrics(StoredMessageBatch batch) - { - MESSAGE_READ_METRIC - .labels(batch.getId().getBookId().getName(), batch.getSessionAlias(), batch.getDirection().getLabel()) - .inc(batch.getMessageCount()); - } - - private static void updateMessageReadMetrics(BookId bookId, StoredGroupedMessageBatch batch) - { - MESSAGE_READ_METRIC - .labels(bookId.getName(), batch.getGroup(), "") - .inc(batch.getMessageCount()); - } - - private static void updateMessageWriteMetrics(MessageBatchEntity entity, BookId bookId) { - MESSAGE_STORE_METRIC - .labels(bookId.getName(), entity.getSessionAlias(), entity.getDirection()) - .inc(entity.getMessageCount()); - - MESSAGE_STORE_UNCOMPRESSED_BYTES - .labels(bookId.getName(), entity.getSessionAlias()) - .inc(entity.getUncompressedContentSize()); - MESSAGE_STORE_COMPRESSED_BYTES - .labels(bookId.getName(), entity.getSessionAlias()) - .inc(entity.getContentSize()); - - } - - private static void updateMessageWriteMetrics(GroupedMessageBatchEntity entity, BookId bookId) { - MESSAGE_STORE_METRIC - .labels(bookId.getName(), entity.getGroup(), "") - .inc(entity.getMessageCount()); - - MESSAGE_STORE_UNCOMPRESSED_BYTES - .labels(bookId.getName(), entity.getGroup()) - .inc(entity.getUncompressedContentSize()); - MESSAGE_STORE_COMPRESSED_BYTES - .labels(bookId.getName(), entity.getGroup()) - .inc(entity.getContentSize()); - } - - public CompletableFuture> getMessageBatches(MessageFilter filter, BookInfo book) - throws CradleStorageException - { - MessageBatchesIteratorProvider provider = - new MessageBatchesIteratorProvider("get messages batches filtered by " + filter, filter, - getOperators(), book, composingService, selectQueryExecutor, - composeReadAttrs(filter.getFetchParameters())); - return provider.nextIterator() - .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); - } - - public CompletableFuture> getGroupedMessageBatches(GroupedMessageFilter filter, - BookInfo book) - throws CradleStorageException - { - GroupedMessageIteratorProvider provider = - new GroupedMessageIteratorProvider("get messages batches filtered by " + filter, filter, - getOperators(), book, composingService, selectQueryExecutor, - composeReadAttrs(filter.getFetchParameters()), filter.getOrder()); - return provider.nextIterator() - .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); - } - - public CompletableFuture> getMessages(MessageFilter filter, BookInfo book) - throws CradleStorageException - { - MessagesIteratorProvider provider = - new MessagesIteratorProvider("get messages filtered by " + filter, filter, - getOperators(), book, composingService, selectQueryExecutor, - composeReadAttrs(filter.getFetchParameters())); - return provider.nextIterator() - .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); - } - - private CompletableFuture getNearestTimeAndSequenceBefore(PageInfo page, - MessageBatchOperator mbOperator, String sessionAlias, String direction, LocalDate messageDate, - LocalTime messageTime, long sequence, Function readAttrs) - { - String queryInfo = format("get nearest time and sequence before %s for page '%s'", - TimeUtils.toInstant(messageDate, messageTime), page.getId().getName()); - return selectQueryExecutor.executeSingleRowResultQuery( - () -> mbOperator.getNearestBatchTimeAndSequenceBefore(page.getId().getBookId().getName(), page.getId().getName(), sessionAlias, - direction, messageDate, messageTime, sequence, readAttrs), Function.identity(), queryInfo) - .thenComposeAsync(row -> - { - if (row != null) - return CompletableFuture.completedFuture(row); - - return CompletableFuture.completedFuture(null); - }, composingService); - } - - public CompletableFuture getMessageBatch(StoredMessageId id, PageId pageId) - { - logger.debug("Getting message batch for message with id '{}'", id); - BookId bookId = pageId.getBookId(); - BookInfo bookInfo; - try - { - bookInfo = getBook(bookId); - } - catch (CradleStorageException e) - { - return CompletableFuture.failedFuture(e); - } - - LocalDateTime ldt = TimeUtils.toLocalTimestamp(id.getTimestamp()); - CassandraOperators operators = getOperators(); - MessageBatchEntityConverter mbEntityConverter = operators.getMessageBatchEntityConverter(); - MessageBatchOperator mbOperator = operators.getMessageBatchOperator(); - - return getNearestTimeAndSequenceBefore(bookInfo.getPage(pageId), mbOperator, id.getSessionAlias(), - id.getDirection().getLabel(), ldt.toLocalDate(), ldt.toLocalTime(), id.getSequence(), readAttrs) - .thenComposeAsync(row -> - { - if (row == null) - { - logger.debug("No message batches found by id '{}'", id); - return CompletableFuture.completedFuture(null); - } - return selectQueryExecutor.executeSingleRowResultQuery( - () -> mbOperator.get(pageId.getBookId().getName(), pageId.getName(), id.getSessionAlias(), - id.getDirection().getLabel(), ldt.toLocalDate(), - row.getLocalTime(FIELD_FIRST_MESSAGE_TIME), row.getLong(FIELD_SEQUENCE), readAttrs), - mbEntityConverter::getEntity, - format("get message batch for message with id '%s'", id)) - .thenApplyAsync(entity -> - { - if (entity == null) - return null; - StoredMessageBatch batch = mapMessageBatchEntity(pageId, entity); - logger.debug("Message batch with id '{}' found for message with id '{}'", - batch.getId(), id); - return batch; - }, composingService); - }, composingService); - } - - public CompletableFuture getMessage(StoredMessageId id, PageId pageId) - { - return getMessageBatch(id, pageId) - .thenComposeAsync(batch -> - { - if (batch == null) - return CompletableFuture.completedFuture(null); - - Optional - found = batch.getMessages().stream().filter(m -> id.equals(m.getId())).findFirst(); - if (found.isPresent()) - return CompletableFuture.completedFuture(found.get()); - - logger.debug("There is no message with id '{}' in batch '{}'", id, batch.getId()); - return CompletableFuture.completedFuture(null); - }, composingService); - } - - private CompletableFuture storePageGroup(GroupedMessageBatchEntity groupedMessageBatchEntity) { - CassandraOperators operators = getOperators(); - PageGroupEntity pageGroupEntity = new PageGroupEntity( - groupedMessageBatchEntity.getBook(), - groupedMessageBatchEntity.getPage(), - groupedMessageBatchEntity.getGroup()); - - if (operators.getPageGroupCache().contains(pageGroupEntity)) { - logger.debug("Skipped writing group '{}' for page '{}'", pageGroupEntity.getGroup(), pageGroupEntity.getPage()); - return CompletableFuture.completedFuture(null); - } - - logger.debug("Writing group '{}' for page '{}'", pageGroupEntity.getGroup(), pageGroupEntity.getPage()); - return operators.getPageGroupsOperator().write(pageGroupEntity, writeAttrs) - .whenCompleteAsync((result, e) -> { - if (e == null) { - operators.getPageGroupCache().store(pageGroupEntity); - } - }, composingService); - } - - private CompletableFuture storeGroup(GroupedMessageBatchEntity groupedMessageBatchEntity) { - CassandraOperators operators = getOperators(); - GroupEntity groupEntity = new GroupEntity(groupedMessageBatchEntity.getBook(), groupedMessageBatchEntity.getGroup()); - - if (operators.getGroupCache().contains(groupEntity)) { - logger.debug("Skipped writing group '{}'", groupEntity.getGroup()); - return CompletableFuture.completedFuture(null); - } - - logger.debug("Writing group '{}'", groupEntity.getGroup()); - return operators.getGroupsOperator().write(groupEntity, writeAttrs) - .whenCompleteAsync((result, e) -> { - if (e == null) { - operators.getGroupCache().store(groupEntity); - } - }, composingService); - } - - public CompletableFuture storePageSession(MessageBatchToStore batch, PageId pageId) - { - StoredMessageId batchId = batch.getId(); - CassandraOperators operators = getOperators(); - CachedPageSession cachedPageSession = new CachedPageSession(pageId.toString(), - batchId.getSessionAlias(), batchId.getDirection().getLabel()); - if (!operators.getPageSessionsCache().store(cachedPageSession)) - { - logger.debug("Skipped writing page/session of message batch '{}'", batchId); - return CompletableFuture.completedFuture(null); - } - - logger.debug("Writing page/session of batch '{}'", batchId); - - return operators.getPageSessionsOperator().write(new PageSessionEntity(batchId, pageId), writeAttrs); - } - - public CompletableFuture storeSession(MessageBatchToStore batch) - { - StoredMessageId batchId = batch.getId(); - BookId bookId = batchId.getBookId(); - CassandraOperators operators = getOperators(); - CachedSession cachedSession = new CachedSession(bookId.toString(), batch.getSessionAlias()); - if (!operators.getSessionsCache().store(cachedSession)) - { - logger.debug("Skipped writing book/session of message batch '{}'", batchId); - return CompletableFuture.completedFuture(null); - } - logger.debug("Writing book/session of batch '{}'", batchId); - - return operators.getSessionsOperator().write(new SessionEntity(bookId.toString(), batch.getSessionAlias()), writeAttrs); - } - - public CompletableFuture storeMessageBatch(MessageBatchToStore batch, PageId pageId) { - BookId bookId = pageId.getBookId(); - MessageBatchOperator mbOperator = getOperators().getMessageBatchOperator(); - - return CompletableFuture.supplyAsync(() -> { - try { - return MessageBatchEntityUtils.toSerializedEntity(batch, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); - } catch (Exception e) { - throw new CompletionException(e); - } - }, composingService).thenCompose(serializedEntity -> { - MessageBatchEntity entity = serializedEntity.getEntity(); - List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); - - return mbOperator.write(entity, writeAttrs) - .thenRunAsync(() -> { - messageStatisticsCollector.updateMessageBatchStatistics(bookId, - entity.getPage(), - entity.getSessionAlias(), - entity.getDirection(), - meta); - sessionStatisticsCollector.updateSessionStatistics(bookId, - entity.getPage(), - SessionRecordType.SESSION, - entity.getSessionAlias(), - meta); - updateMessageWriteMetrics(entity, bookId); - }, composingService); - }); - } - - public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToStore batchToStore, PageId pageId) { - BookId bookId = pageId.getBookId(); - GroupedMessageBatchOperator gmbOperator = getOperators().getGroupedMessageBatchOperator(); - - return CompletableFuture.supplyAsync(() -> { - try { - return GroupedMessageEntityUtils.toSerializedEntity(batchToStore, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); - } catch (Exception e) { - throw new CompletionException(e); - } - }, composingService).thenCompose(serializedEntity -> { - GroupedMessageBatchEntity entity = serializedEntity.getEntity(); - List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); - - return gmbOperator.write(entity, writeAttrs) - .thenRunAsync(() -> storePageGroup(entity), composingService) - .thenRunAsync(() -> storeGroup(entity), composingService) - .thenRunAsync(() -> { - messageStatisticsCollector.updateMessageBatchStatistics(bookId, - pageId.getName(), - entity.getGroup(), - "", - meta); - sessionStatisticsCollector.updateSessionStatistics(bookId, - pageId.getName(), - SessionRecordType.SESSION_GROUP, - entity.getGroup(), - meta); - updateMessageWriteMetrics(entity, bookId); - }, composingService); - }); - } - - - public long getBoundarySequence(String sessionAlias, Direction direction, BookInfo book, boolean first) - throws CradleStorageException - { - MessageBatchOperator mbOp = getOperators().getMessageBatchOperator(); - PageInfo currentPage = first ? book.getFirstPage() : book.getLastPage(); - Row row = null; - - while (row == null && currentPage != null) - { - String page = currentPage.getId().getName(); - String bookName = book.getId().getName(); - String queryInfo = format("get %s sequence for book '%s' page '%s', session alias '%s', " + - "direction '%s'", (first ? "first" : "last"), bookName, page, sessionAlias, direction); - try - { - row = selectQueryExecutor.executeSingleRowResultQuery( - () -> first ? mbOp.getFirstSequence(bookName, page, sessionAlias, direction.getLabel(), readAttrs) - : mbOp.getLastSequence(bookName, page, sessionAlias, direction.getLabel(), readAttrs), - Function.identity(), queryInfo).get(); - } - catch (InterruptedException | ExecutionException e) - { - throw new CradleStorageException("Error occurs while " + queryInfo, e); - } - - if (row == null) - currentPage = first ? book.getNextPage(currentPage.getStarted()) - : book.getPreviousPage(currentPage.getStarted()); - } - if (row == null) - { - logger.debug("There is no messages yet in book '{}' with session alias '{}' and direction '{}'", - book.getId(), sessionAlias, direction); - return EMPTY_MESSAGE_INDEX; - } - - return row.getLong(first ? FIELD_SEQUENCE : FIELD_LAST_SEQUENCE); - } +import static java.util.Objects.requireNonNull; + +public class MessagesWorker extends Worker { + private static final Logger logger = LoggerFactory.getLogger(MessagesWorker.class); + + private static final MetricHolder MESSAGE_READ_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_message_read_total") + .help("Fetched messages") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder MESSAGE_BATCH_READ_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_message_batch_read_total") + .help("Fetched message batches") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder MESSAGE_STORE_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_message_stored_total") + .help("Stored messages") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder MESSAGE_BATCH_STORE_METRIC = new MetricHolder<>( + Counter.build() + .name("cradle_message_batch_stored_total") + .help("Stored message batches") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder MESSAGE_STORE_UNCOMPRESSED_BYTES = new MetricHolder<>( + Counter.build() + .name("cradle_message_stored_uncompressed_bytes_total") + .help("Stored uncompressed message bytes") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + private static final MetricHolder MESSAGE_STORE_COMPRESSED_BYTES = new MetricHolder<>( + Counter.build() + .name("cradle_message_stored_compressed_bytes_total") + .help("Stored compressed message bytes") + .labelNames(StreamLabel.LABEL_NAMES) + .register() + ); + + private final MessageStatisticsCollector messageStatisticsCollector; + private final SessionStatisticsCollector sessionStatisticsCollector; + + public MessagesWorker( + WorkerSupplies workerSupplies + , MessageStatisticsCollector messageStatisticsCollector + , SessionStatisticsCollector sessionStatisticsCollector + ) { + super(workerSupplies); + this.messageStatisticsCollector = messageStatisticsCollector; + this.sessionStatisticsCollector = sessionStatisticsCollector; + } + + public static StoredMessageBatch mapMessageBatchEntity(PageId pageId, MessageBatchEntity entity) { + try { + StoredMessageBatch batch = MessageBatchEntityUtils.toStoredMessageBatch(entity, pageId); + updateMessageReadMetrics(batch); + return batch; + } catch (DataFormatException | IOException | CompressException e) { + throw new CompletionException("Error while converting message batch entity into stored message batch", e); + } + } + + public static StoredGroupedMessageBatch mapGroupedMessageBatchEntity(PageId pageId, GroupedMessageBatchEntity entity) { + try { + StoredGroupedMessageBatch batch = GroupedMessageEntityUtils.toStoredGroupedMessageBatch(entity, pageId); + updateMessageReadMetrics(pageId.getBookId(), batch); + return batch; + } catch (DataFormatException | IOException | CompressException e) { + throw new CompletionException("Error while converting message batch entity into stored message batch", e); + } + } + + private static void updateMessageReadMetrics(StoredMessageBatch batch) { + StreamLabel key = new StreamLabel( + batch.getId().getBookId().getName(), + batch.getSessionAlias(), + batch.getDirection().getLabel()); + MESSAGE_READ_METRIC.inc(key, batch.getMessageCount()); + MESSAGE_BATCH_READ_METRIC.inc(key); + } + + private static void updateMessageReadMetrics(BookId bookId, StoredGroupedMessageBatch batch) { + StreamLabel key = new StreamLabel( + bookId.getName(), + batch.getGroup()); + MESSAGE_READ_METRIC.inc(key, batch.getMessageCount()); + MESSAGE_BATCH_READ_METRIC.inc(key); + } + + private static void updateMessageWriteMetrics(MessageBatchEntity entity, BookId bookId) { + StreamLabel key = new StreamLabel(bookId.getName(), entity.getSessionAlias(), entity.getDirection()); + MESSAGE_STORE_METRIC.inc(key, entity.getMessageCount()); + MESSAGE_BATCH_STORE_METRIC.inc(key); + MESSAGE_STORE_UNCOMPRESSED_BYTES.inc(key, entity.getUncompressedContentSize()); + MESSAGE_STORE_COMPRESSED_BYTES.inc(key, entity.getContentSize()); + + } + + private static void updateMessageWriteMetrics(GroupedMessageBatchEntity entity, BookId bookId) { + StreamLabel key = new StreamLabel(bookId.getName(), entity.getGroup()); + MESSAGE_STORE_METRIC.inc(key, entity.getMessageCount()); + MESSAGE_BATCH_STORE_METRIC.inc(key); + MESSAGE_STORE_UNCOMPRESSED_BYTES.inc(key, entity.getUncompressedContentSize()); + MESSAGE_STORE_COMPRESSED_BYTES.inc(key, entity.getContentSize()); + } + + public CompletableFuture> getMessageBatches(MessageFilter filter, BookInfo book) + throws CradleStorageException { + MessageBatchesIteratorProvider provider = + new MessageBatchesIteratorProvider("get messages batches filtered by " + filter, filter, + getOperators(), book, composingService, selectQueryExecutor, + composeReadAttrs(filter.getFetchParameters())); + return provider.nextIterator() + .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); + } + + public CompletableFuture> getGroupedMessageBatches( + GroupedMessageFilter filter, + BookInfo book + ) + throws CradleStorageException { + GroupedMessageIteratorProvider provider = + new GroupedMessageIteratorProvider("get messages batches filtered by " + filter, filter, + getOperators(), book, composingService, selectQueryExecutor, + composeReadAttrs(filter.getFetchParameters()), filter.getOrder()); + return provider.nextIterator() + .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); + } + + public CompletableFuture> getMessages(MessageFilter filter, BookInfo book) + throws CradleStorageException { + MessagesIteratorProvider provider = + new MessagesIteratorProvider("get messages filtered by " + filter, filter, + getOperators(), book, composingService, selectQueryExecutor, + composeReadAttrs(filter.getFetchParameters())); + return provider.nextIterator() + .thenApplyAsync(r -> new CassandraCradleResultSet<>(r, provider), composingService); + } + + private CompletableFuture getNearestTimeAndSequenceBefore( + PageInfo page, + MessageBatchOperator mbOperator, String sessionAlias, String direction, LocalDate messageDate, + LocalTime messageTime, long sequence, Function readAttrs + ) { + String queryInfo = format("get nearest time and sequence before %s for page '%s'", + TimeUtils.toInstant(messageDate, messageTime), page.getId().getName()); + return selectQueryExecutor.executeSingleRowResultQuery( + () -> mbOperator.getNearestBatchTimeAndSequenceBefore(page.getId().getBookId().getName(), page.getId().getName(), sessionAlias, + direction, messageDate, messageTime, sequence, readAttrs), Function.identity(), queryInfo) + .thenComposeAsync(row -> + { + if (row != null) + return CompletableFuture.completedFuture(row); + + return CompletableFuture.completedFuture(null); + }, composingService); + } + + public CompletableFuture getMessageBatch(StoredMessageId id, PageId pageId) { + logger.debug("Getting message batch for message with id '{}'", id); + BookId bookId = pageId.getBookId(); + BookInfo bookInfo; + try { + bookInfo = getBook(bookId); + } catch (CradleStorageException e) { + return CompletableFuture.failedFuture(e); + } + + LocalDateTime ldt = TimeUtils.toLocalTimestamp(id.getTimestamp()); + CassandraOperators operators = getOperators(); + MessageBatchEntityConverter mbEntityConverter = operators.getMessageBatchEntityConverter(); + MessageBatchOperator mbOperator = operators.getMessageBatchOperator(); + + return getNearestTimeAndSequenceBefore(bookInfo.getPage(pageId), mbOperator, id.getSessionAlias(), + id.getDirection().getLabel(), ldt.toLocalDate(), ldt.toLocalTime(), id.getSequence(), readAttrs) + .thenComposeAsync(row -> + { + if (row == null) { + logger.debug("No message batches found by id '{}'", id); + return CompletableFuture.completedFuture(null); + } + return selectQueryExecutor.executeSingleRowResultQuery( + () -> mbOperator.get(pageId.getBookId().getName(), pageId.getName(), id.getSessionAlias(), + id.getDirection().getLabel(), ldt.toLocalDate(), + row.getLocalTime(FIELD_FIRST_MESSAGE_TIME), row.getLong(FIELD_SEQUENCE), readAttrs), + mbEntityConverter::getEntity, + format("get message batch for message with id '%s'", id)) + .thenApplyAsync(entity -> + { + if (entity == null) + return null; + StoredMessageBatch batch = mapMessageBatchEntity(pageId, entity); + logger.debug("Message batch with id '{}' found for message with id '{}'", + batch.getId(), id); + return batch; + }, composingService); + }, composingService); + } + + public CompletableFuture getMessage(StoredMessageId id, PageId pageId) { + return getMessageBatch(id, pageId) + .thenComposeAsync(batch -> + { + if (batch == null) + return CompletableFuture.completedFuture(null); + + Optional + found = batch.getMessages().stream().filter(m -> id.equals(m.getId())).findFirst(); + if (found.isPresent()) + return CompletableFuture.completedFuture(found.get()); + + logger.debug("There is no message with id '{}' in batch '{}'", id, batch.getId()); + return CompletableFuture.completedFuture(null); + }, composingService); + } + + private CompletableFuture storePageGroup(GroupedMessageBatchEntity groupedMessageBatchEntity) { + CassandraOperators operators = getOperators(); + PageGroupEntity pageGroupEntity = new PageGroupEntity( + groupedMessageBatchEntity.getBook(), + groupedMessageBatchEntity.getPage(), + groupedMessageBatchEntity.getGroup()); + + if (operators.getPageGroupCache().contains(pageGroupEntity)) { + logger.debug("Skipped writing group '{}' for page '{}'", pageGroupEntity.getGroup(), pageGroupEntity.getPage()); + return CompletableFuture.completedFuture(null); + } + + logger.debug("Writing group '{}' for page '{}'", pageGroupEntity.getGroup(), pageGroupEntity.getPage()); + return operators.getPageGroupsOperator().write(pageGroupEntity, writeAttrs) + .whenCompleteAsync((result, e) -> { + if (e == null) { + operators.getPageGroupCache().store(pageGroupEntity); + } + }, composingService); + } + + private CompletableFuture storeGroup(GroupedMessageBatchEntity groupedMessageBatchEntity) { + CassandraOperators operators = getOperators(); + GroupEntity groupEntity = new GroupEntity(groupedMessageBatchEntity.getBook(), groupedMessageBatchEntity.getGroup()); + + if (operators.getGroupCache().contains(groupEntity)) { + logger.debug("Skipped writing group '{}'", groupEntity.getGroup()); + return CompletableFuture.completedFuture(null); + } + + logger.debug("Writing group '{}'", groupEntity.getGroup()); + return operators.getGroupsOperator().write(groupEntity, writeAttrs) + .whenCompleteAsync((result, e) -> { + if (e == null) { + operators.getGroupCache().store(groupEntity); + } + }, composingService); + } + + public CompletableFuture storePageSession(MessageBatchToStore batch, PageId pageId) { + StoredMessageId batchId = batch.getId(); + CassandraOperators operators = getOperators(); + CachedPageSession cachedPageSession = new CachedPageSession(pageId.toString(), + batchId.getSessionAlias(), batchId.getDirection().getLabel()); + if (!operators.getPageSessionsCache().store(cachedPageSession)) { + logger.debug("Skipped writing page/session of message batch '{}'", batchId); + return CompletableFuture.completedFuture(null); + } + + logger.debug("Writing page/session of batch '{}'", batchId); + + return operators.getPageSessionsOperator().write(new PageSessionEntity(batchId, pageId), writeAttrs); + } + + public CompletableFuture storeSession(MessageBatchToStore batch) { + StoredMessageId batchId = batch.getId(); + BookId bookId = batchId.getBookId(); + CassandraOperators operators = getOperators(); + CachedSession cachedSession = new CachedSession(bookId.toString(), batch.getSessionAlias()); + if (!operators.getSessionsCache().store(cachedSession)) { + logger.debug("Skipped writing book/session of message batch '{}'", batchId); + return CompletableFuture.completedFuture(null); + } + logger.debug("Writing book/session of batch '{}'", batchId); + + return operators.getSessionsOperator().write(new SessionEntity(bookId.toString(), batch.getSessionAlias()), writeAttrs); + } + + public CompletableFuture storeMessageBatch(MessageBatchToStore batch, PageId pageId) { + BookId bookId = pageId.getBookId(); + MessageBatchOperator mbOperator = getOperators().getMessageBatchOperator(); + + return CompletableFuture.supplyAsync(() -> { + try { + return MessageBatchEntityUtils.toSerializedEntity(batch, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); + } catch (Exception e) { + throw new CompletionException(e); + } + }, composingService).thenCompose(serializedEntity -> { + MessageBatchEntity entity = serializedEntity.getEntity(); + List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); + + return mbOperator.write(entity, writeAttrs) + .thenRunAsync(() -> { + messageStatisticsCollector.updateMessageBatchStatistics(bookId, + entity.getPage(), + entity.getSessionAlias(), + entity.getDirection(), + meta); + sessionStatisticsCollector.updateSessionStatistics(bookId, + entity.getPage(), + SessionRecordType.SESSION, + entity.getSessionAlias(), + meta); + updateMessageWriteMetrics(entity, bookId); + }, composingService); + }); + } + + public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToStore batchToStore, PageId pageId) { + BookId bookId = pageId.getBookId(); + GroupedMessageBatchOperator gmbOperator = getOperators().getGroupedMessageBatchOperator(); + + return CompletableFuture.supplyAsync(() -> { + try { + return GroupedMessageEntityUtils.toSerializedEntity(batchToStore, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); + } catch (Exception e) { + throw new CompletionException(e); + } + }, composingService).thenCompose(serializedEntity -> { + GroupedMessageBatchEntity entity = serializedEntity.getEntity(); + List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); + + return gmbOperator.write(entity, writeAttrs) + .thenComposeAsync((value) -> CompletableFuture.allOf( + storePageGroup(entity), + storeGroup(entity), + CompletableFuture.runAsync(() -> { + messageStatisticsCollector.updateMessageBatchStatistics(bookId, + pageId.getName(), + entity.getGroup(), + "", + meta); + sessionStatisticsCollector.updateSessionStatistics(bookId, + pageId.getName(), + SessionRecordType.SESSION_GROUP, + entity.getGroup(), + meta); + updateMessageWriteMetrics(entity, bookId); + }, composingService) + ), composingService); + }); + } + + + public long getBoundarySequence(String sessionAlias, Direction direction, BookInfo book, boolean first) + throws CradleStorageException { + MessageBatchOperator mbOp = getOperators().getMessageBatchOperator(); + PageInfo currentPage = first ? book.getFirstPage() : book.getLastPage(); + Row row = null; + + while (row == null && currentPage != null) { + String page = currentPage.getId().getName(); + String bookName = book.getId().getName(); + String queryInfo = format("get %s sequence for book '%s' page '%s', session alias '%s', " + + "direction '%s'", (first ? "first" : "last"), bookName, page, sessionAlias, direction); + try { + row = selectQueryExecutor.executeSingleRowResultQuery( + () -> first ? mbOp.getFirstSequence(bookName, page, sessionAlias, direction.getLabel(), readAttrs) + : mbOp.getLastSequence(bookName, page, sessionAlias, direction.getLabel(), readAttrs), + Function.identity(), queryInfo).get(); + } catch (InterruptedException | ExecutionException e) { + throw new CradleStorageException("Error occurs while " + queryInfo, e); + } + + if (row == null) + currentPage = first ? book.getNextPage(currentPage.getStarted()) + : book.getPreviousPage(currentPage.getStarted()); + } + if (row == null) { + logger.debug("There is no messages yet in book '{}' with session alias '{}' and direction '{}'", + book.getId(), sessionAlias, direction); + return EMPTY_MESSAGE_INDEX; + } + + return row.getLong(first ? FIELD_SEQUENCE : FIELD_LAST_SEQUENCE); + } + + private static class StreamLabel implements MetricHolder.LabelHolder { + private static final String[] LABEL_NAMES = new String[]{BOOK_ID, SESSION_ALIAS_OR_GROUP, DIRECTION}; + private final String bookId; + private final String sessionAliasOrGroup; + private final String direction; + private final int hash; + + private StreamLabel( + @Nonnull String bookId, + @Nonnull String sessionAlias, + @Nonnull String direction + ) { + this.bookId = requireNonNull(bookId, "'bookId' can't be null"); + this.sessionAliasOrGroup = requireNonNull(sessionAlias, "'sessionAliasOrGroup' can't be null"); + this.direction = requireNonNull(direction, "'direction' can't be null"); + this.hash = Objects.hash(bookId, sessionAliasOrGroup, direction); + } + + private StreamLabel( + @Nonnull String bookId, + @Nonnull String sessionGroup + ) { + this(bookId, sessionGroup, ""); + } + + @Override + public String[] getLabels() { + return new String[]{bookId, sessionAliasOrGroup, direction}; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + StreamLabel that = (StreamLabel) o; + return Objects.equals(bookId, that.bookId) + && Objects.equals(sessionAliasOrGroup, that.sessionAliasOrGroup) + && Objects.equals(direction, that.direction); + } + + @Override + public int hashCode() { + return hash; + } + } } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MetricHolder.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MetricHolder.java new file mode 100644 index 000000000..62108bb9c --- /dev/null +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MetricHolder.java @@ -0,0 +1,46 @@ +/* + * Copyright 2023 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.cradle.cassandra.workers; + +import io.prometheus.client.Counter; + +import javax.annotation.Nonnull; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class MetricHolder { + + private final ConcurrentMap map = new ConcurrentHashMap<>(); + private final Counter counter; + + public MetricHolder(@Nonnull Counter counter) { + this.counter = Objects.requireNonNull(counter, "'counter' can't be null"); + } + + public void inc(T key, double value) { + map.computeIfAbsent(key, a -> counter.labels(a.getLabels())).inc(value); + } + + public void inc(T key) { + inc(key, 1); + } + + public interface LabelHolder { + String[] getLabels(); + } +} diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/Worker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/Worker.java index d53779cd2..6b0727ec5 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/Worker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/Worker.java @@ -31,45 +31,45 @@ import java.util.function.Function; public abstract class Worker { - // Metric Labels - public static final String BOOK_ID = "book_id"; - public static final String SESSION_ALIAS = "session_alias"; - public static final String SCOPE = "scope"; - public static final String DIRECTION = "direction"; + // Metric Labels + public static final String BOOK_ID = "book_id"; + public static final String SESSION_ALIAS_OR_GROUP = "session_alias_or_group"; + public static final String SCOPE = "scope"; + public static final String DIRECTION = "direction"; - protected final CassandraStorageSettings settings; - private final CassandraOperators operators; - protected final ExecutorService composingService; - protected final BookCache bookCache; - protected final SelectQueryExecutor selectQueryExecutor; - protected final Function writeAttrs, - readAttrs; + protected final CassandraStorageSettings settings; + private final CassandraOperators operators; + protected final ExecutorService composingService; + protected final BookCache bookCache; + protected final SelectQueryExecutor selectQueryExecutor; + protected final Function writeAttrs, + readAttrs; - public Worker(WorkerSupplies workerSupplies) { - this.settings = workerSupplies.getSettings(); - this.operators = workerSupplies.getOperators(); - this.composingService = workerSupplies.getComposingService(); - this.bookCache = workerSupplies.getBookCache(); - this.selectQueryExecutor = workerSupplies.getSelectExecutor(); - this.writeAttrs = workerSupplies.getWriteAttrs(); - this.readAttrs = workerSupplies.getReadAttrs(); - } + public Worker(WorkerSupplies workerSupplies) { + this.settings = workerSupplies.getSettings(); + this.operators = workerSupplies.getOperators(); + this.composingService = workerSupplies.getComposingService(); + this.bookCache = workerSupplies.getBookCache(); + this.selectQueryExecutor = workerSupplies.getSelectExecutor(); + this.writeAttrs = workerSupplies.getWriteAttrs(); + this.readAttrs = workerSupplies.getReadAttrs(); + } - protected CassandraOperators getOperators() { - return operators; - } + protected CassandraOperators getOperators() { + return operators; + } - protected BookInfo getBook(BookId bookId) throws CradleStorageException { - return bookCache.getBook(bookId); - } + protected BookInfo getBook(BookId bookId) throws CradleStorageException { + return bookCache.getBook(bookId); + } - protected Function composeReadAttrs(FetchParameters fetchParams) { - if (fetchParams == null) - return readAttrs; + protected Function composeReadAttrs(FetchParameters fetchParams) { + if (fetchParams == null) + return readAttrs; - int fetchSize = fetchParams.getFetchSize(); - long timeout = fetchParams.getTimeout(); - return readAttrs.andThen(builder -> fetchSize > 0 ? builder.setPageSize(fetchSize) : builder) - .andThen(builder -> timeout > 0 ? builder.setTimeout(Duration.ofMillis(timeout)) : builder); - } + int fetchSize = fetchParams.getFetchSize(); + long timeout = fetchParams.getTimeout(); + return readAttrs.andThen(builder -> fetchSize > 0 ? builder.setPageSize(fetchSize) : builder) + .andThen(builder -> timeout > 0 ? builder.setTimeout(Duration.ofMillis(timeout)) : builder); + } } From 7fe8abbedcc710d81a34e2788ffa3631a47158b5 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Tue, 6 Jun 2023 20:22:37 +0400 Subject: [PATCH 22/31] reverted CassandraConsistencyLevel --- .../cassandra/CassandraCradleStorage.java | 12 +++++------ .../cassandra/CassandraStorageSettings.java | 21 +++++++++---------- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java index 3637791f3..1bed9a2d8 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java @@ -141,7 +141,7 @@ protected void doInit(boolean prepareStorage) throws CradleStorageException { DriverMetrics.register(connection.getSession()); exec = new QueryExecutor(connection.getSession(), - settings.getTimeout(), settings.getWriteConsistencyLevel(), settings.getReadConsistencyLevel()); + settings.getTimeout(), settings.getWriteConsistencyLevel().getValue(), settings.getReadConsistencyLevel().getValue()); selectExecutor = new SelectQueryExecutor(connection.getSession(), composingService, multiRowResultExecPolicy, singleRowResultExecPolicy); if (prepareStorage) @@ -152,11 +152,11 @@ protected void doInit(boolean prepareStorage) throws CradleStorageException Duration timeout = Duration.ofMillis(settings.getTimeout()); int resultPageSize = settings.getResultPageSize(); - writeAttrs = builder -> builder.setConsistencyLevel(settings.getWriteConsistencyLevel()) + writeAttrs = builder -> builder.setConsistencyLevel(settings.getWriteConsistencyLevel().getValue()) .setTimeout(timeout); - batchWriteAttrs = builder -> builder.setConsistencyLevel(settings.getWriteConsistencyLevel()) + batchWriteAttrs = builder -> builder.setConsistencyLevel(settings.getWriteConsistencyLevel().getValue()) .setTimeout(timeout); - readAttrs = builder -> builder.setConsistencyLevel(settings.getReadConsistencyLevel()) + readAttrs = builder -> builder.setConsistencyLevel(settings.getReadConsistencyLevel().getValue()) .setTimeout(timeout) .setPageSize(resultPageSize); strictReadAttrs = builder -> builder.setConsistencyLevel(ConsistencyLevel.ALL) @@ -794,7 +794,7 @@ protected CompletableFuture doGetMessageCountAsync(BookId bookId, sum = sum.incrementedBy(res.next().getCounter()); } } catch (CradleStorageException | IOException e) { - LOGGER.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); + LOGGER.error("Error while getting {}", queryInfo, e); } } @@ -844,7 +844,7 @@ protected CompletableFuture doGetCountAsync(BookId bookId, EntityType e sum = sum.incrementedBy(res.next().getCounter()); } } catch (CradleStorageException | IOException e) { - LOGGER.error("Error while getting {}, cause - {}", queryInfo, e.getCause()); + LOGGER.error("Error while getting {}", queryInfo, e); } } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index dbc194d80..80f26d687 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -16,7 +16,6 @@ package com.exactpro.cradle.cassandra; -import com.datastax.oss.driver.api.core.ConsistencyLevel; import com.exactpro.cradle.CoreStorageSettings; import com.exactpro.cradle.cassandra.connection.NetworkTopologyStrategy; import com.exactpro.cradle.cassandra.retries.SelectExecutionPolicy; @@ -33,7 +32,7 @@ public class CassandraStorageSettings extends CoreStorageSettings { public static final String SCHEMA_VERSION = "5.3.0"; - public static final ConsistencyLevel DEFAULT_CONSISTENCY_LEVEL = ConsistencyLevel.LOCAL_QUORUM; + public static final CassandraConsistencyLevel DEFAULT_CONSISTENCY_LEVEL = CassandraConsistencyLevel.LOCAL_QUORUM; public static final int DEFAULT_KEYSPACE_REPL_FACTOR = 1; public static final int DEFAULT_MAX_PARALLEL_QUERIES = 500; public static final int DEFAULT_RESULT_PAGE_SIZE = 0; @@ -57,9 +56,9 @@ public class CassandraStorageSettings extends CoreStorageSettings { private NetworkTopologyStrategy networkTopologyStrategy; private long timeout = DEFAULT_TIMEOUT; @JsonIgnore - private ConsistencyLevel writeConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; + private CassandraConsistencyLevel writeConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; @JsonIgnore - private ConsistencyLevel readConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; + private CassandraConsistencyLevel readConsistencyLevel = DEFAULT_CONSISTENCY_LEVEL; private String keyspace; private String schemaVersion = SCHEMA_VERSION; private int keyspaceReplicationFactor = DEFAULT_KEYSPACE_REPL_FACTOR; @@ -94,15 +93,15 @@ public CassandraStorageSettings() { public CassandraStorageSettings( long timeout, - ConsistencyLevel writeConsistencyLevel, - ConsistencyLevel readConsistencyLevel + CassandraConsistencyLevel writeConsistencyLevel, + CassandraConsistencyLevel readConsistencyLevel ) { this(null, timeout, writeConsistencyLevel, readConsistencyLevel); } public CassandraStorageSettings( NetworkTopologyStrategy networkTopologyStrategy, long timeout, - ConsistencyLevel writeConsistencyLevel, ConsistencyLevel readConsistencyLevel + CassandraConsistencyLevel writeConsistencyLevel, CassandraConsistencyLevel readConsistencyLevel ) { this(); this.networkTopologyStrategy = networkTopologyStrategy; @@ -164,19 +163,19 @@ public void setTimeout(long timeout) { this.timeout = timeout; } - public ConsistencyLevel getWriteConsistencyLevel() { + public CassandraConsistencyLevel getWriteConsistencyLevel() { return writeConsistencyLevel; } - public void setWriteConsistencyLevel(ConsistencyLevel writeConsistencyLevel) { + public void setWriteConsistencyLevel(CassandraConsistencyLevel writeConsistencyLevel) { this.writeConsistencyLevel = writeConsistencyLevel; } - public ConsistencyLevel getReadConsistencyLevel() { + public CassandraConsistencyLevel getReadConsistencyLevel() { return readConsistencyLevel; } - public void setReadConsistencyLevel(ConsistencyLevel readConsistencyLevel) { + public void setReadConsistencyLevel(CassandraConsistencyLevel readConsistencyLevel) { this.readConsistencyLevel = readConsistencyLevel; } From a60b3299a6ed7a8f3cec2304fbaa7853ff8e98d8 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Tue, 6 Jun 2023 20:44:43 +0400 Subject: [PATCH 23/31] Reverted backward compatible --- .../cradle/testevents/TestEventToStore.java | 38 ++++++++++++++----- 1 file changed, 28 insertions(+), 10 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java index f49d78418..f24f506f3 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java @@ -1,17 +1,17 @@ /* - * Copyright 2020-2023 Exactpro (Exactpro Systems Limited) + * Copyright 2023 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 + * 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 + * 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. + * 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.cradle.testevents; @@ -21,6 +21,8 @@ import java.time.Instant; +import static com.exactpro.cradle.CoreStorageSettings.DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; + /** * Holds basic information about test event prepared to be stored in Cradle. Events extend this class with additional data */ @@ -39,6 +41,22 @@ public TestEventToStore(StoredTestEventId id, String name, StoredTestEventId par TestEventUtils.validateTestEvent(this, storeActionRejectionThreshold); } + public static TestEventSingleToStoreBuilder singleBuilder(long storeActionRejectionThreshold) { + return new TestEventSingleToStoreBuilder(storeActionRejectionThreshold); + } + + public static TestEventSingleToStoreBuilder singleBuilder() { + return singleBuilder(DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS); + } + + public static TestEventBatchToStoreBuilder batchBuilder(int maxBatchSize, long storeActionRejectionThreshold) { + return new TestEventBatchToStoreBuilder(maxBatchSize, storeActionRejectionThreshold); + } + + public static TestEventBatchToStoreBuilder batchBuilder(int maxBatchSize) { + return batchBuilder(maxBatchSize, DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS); + } + @Override public StoredTestEventId getId() { return id; From c2f76234e7fdfba0c836e4b7d1f8181146e2e9b1 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 7 Jun 2023 13:52:10 +0400 Subject: [PATCH 24/31] Reverted copyright --- .../cradle/testevents/TestEventToStore.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java b/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java index f24f506f3..1005e263f 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/testevents/TestEventToStore.java @@ -1,17 +1,17 @@ /* - * Copyright 2023 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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 + * 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 + * 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. + * 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.cradle.testevents; From 1a6709eca55c747c4d894f8cf9ae6a74e90bb3d6 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 14 Jun 2023 12:58:10 +0400 Subject: [PATCH 25/31] Corrected build gradle * removed suppressed vulnerability from log --- build.gradle | 324 +++++++++++++++++----------------- cradle-cassandra/build.gradle | 62 ++++--- 2 files changed, 195 insertions(+), 191 deletions(-) diff --git a/build.gradle b/build.gradle index 9d86ab9e7..00f858c07 100644 --- a/build.gradle +++ b/build.gradle @@ -1,183 +1,183 @@ plugins { - id "io.github.gradle-nexus.publish-plugin" version"1.0.0" - id "org.owasp.dependencycheck" version "8.1.2" - id 'signing' + id "io.github.gradle-nexus.publish-plugin" version "1.0.0" + id "org.owasp.dependencycheck" version "8.2.1" + id 'signing' } ext { - sharedDir = file("${project.rootDir}/shared") + sharedDir = file("${project.rootDir}/shared") } allprojects { - version = release_version - group = 'com.exactpro.th2' - - dependencyCheck { - formats=['SARIF', 'JSON', 'HTML'] - failBuildOnCVSS=5 - suppressionFile="suppressions.xml" - analyzers { - assemblyEnabled = false - nugetconfEnabled = false - nodeEnabled = false - } - } + version = release_version + group = 'com.exactpro.th2' + + dependencyCheck { + formats = ['SARIF', 'JSON', 'HTML'] + failBuildOnCVSS = 5 + analyzers { + assemblyEnabled = false + nugetconfEnabled = false + nodeEnabled = false + } + } } subprojects { - apply plugin: 'java-library' - apply plugin: 'maven-publish' - apply plugin: 'signing' - apply plugin: 'org.owasp.dependencycheck' - - repositories { - mavenCentral() - maven { - name 'Sonatype_snapshots' - url 'https://s01.oss.sonatype.org/content/repositories/snapshots/' - } - - maven { - name 'Sonatype_releases' - url 'https://s01.oss.sonatype.org/content/repositories/releases/' - } - configurations.all { - resolutionStrategy.cacheChangingModulesFor 0, 'seconds' - resolutionStrategy.cacheDynamicVersionsFor 0, 'seconds' - } - } - - - defaultTasks 'build' - - sourceCompatibility = JavaVersion.VERSION_11 //Java version compatibility to use when compiling Java source. - targetCompatibility = JavaVersion.VERSION_11 //Java version to generate classes for. - compileJava.options.debugOptions.debugLevel = 'source,lines,vars' // Include debug information - - jar { - manifest { - attributes( - 'Created-By': "${System.getProperty('java.version')} (${System.getProperty('java.vendor')})", - 'Specification-Title': '', - 'Specification-Vendor': 'Exactpro Systems LLC', - 'Implementation-Title': project.archivesBaseName, - 'Implementation-Vendor': 'Exactpro Systems LLC', - 'Implementation-Vendor-Id': 'com.exactpro', - 'Implementation-Version': project.version, - 'Implementation-Version-Date': getDate() - ) - } - } - - java { - withJavadocJar() - withSourcesJar() - } - - // conditionals for publications - tasks.withType(PublishToMavenRepository) { - onlyIf { - (repository == publishing.repositories.nexusRepository && - project.hasProperty('nexus_user') && - project.hasProperty('nexus_password') && - project.hasProperty('nexus_url')) || - (repository == publishing.repositories.sonatype && - project.hasProperty('sonatypeUsername') && - project.hasProperty('sonatypePassword')) || - (repository == publishing.repositories.localRepo) - } - } - tasks.withType(Sign) { - onlyIf { project.hasProperty('signingKey') && - project.hasProperty('signingPassword') - } - } - // disable running task 'initializeSonatypeStagingRepository' on a gitlab - tasks.whenTaskAdded {task -> - if(task.name.equals('initializeSonatypeStagingRepository') && - !(project.hasProperty('sonatypeUsername') && project.hasProperty('sonatypePassword')) - ) { - task.enabled = false - } - } - - publishing { - publications { - mavenJava(MavenPublication) { - from(components.java) - pom { - name = project.name - packaging = 'jar' - description = rootProject.description - url = vcs_url - scm { - url = vcs_url - } - licenses { - license { - name = 'The Apache License, Version 2.0' - url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' - } - } - developers { - developer { - id = 'developer' - name = 'developer' - email = 'developer@exactpro.com' - } - } - scm { - url = vcs_url - } - } - } - } - repositories { - maven { - name = 'localRepo' - url = sharedDir - } - //Nexus repo to publish from gitlab - maven { - name = 'nexusRepository' - credentials { - username = project.findProperty('nexus_user') - password = project.findProperty('nexus_password') - } - url = project.findProperty('nexus_url') - } - } - } - - signing { - def signingKey = findProperty("signingKey") - def signingPassword = findProperty("signingPassword") - useInMemoryPgpKeys(signingKey, signingPassword) - sign publishing.publications.mavenJava - } + apply plugin: 'java-library' + apply plugin: 'maven-publish' + apply plugin: 'signing' + apply plugin: 'org.owasp.dependencycheck' + + repositories { + mavenCentral() + maven { + name 'Sonatype_snapshots' + url 'https://s01.oss.sonatype.org/content/repositories/snapshots/' + } + + maven { + name 'Sonatype_releases' + url 'https://s01.oss.sonatype.org/content/repositories/releases/' + } + configurations.configureEach { + resolutionStrategy.cacheChangingModulesFor 0, 'seconds' + resolutionStrategy.cacheDynamicVersionsFor 0, 'seconds' + } + } + + + defaultTasks 'build' + + sourceCompatibility = JavaVersion.VERSION_11 //Java version compatibility to use when compiling Java source. + targetCompatibility = JavaVersion.VERSION_11 //Java version to generate classes for. + compileJava.options.debugOptions.debugLevel = 'source,lines,vars' // Include debug information + + jar { + manifest { + attributes( + 'Created-By': "${System.getProperty('java.version')} (${System.getProperty('java.vendor')})", + 'Specification-Title': '', + 'Specification-Vendor': 'Exactpro Systems LLC', + 'Implementation-Title': project.archivesBaseName, + 'Implementation-Vendor': 'Exactpro Systems LLC', + 'Implementation-Vendor-Id': 'com.exactpro', + 'Implementation-Version': project.version, + 'Implementation-Version-Date': getDate() + ) + } + } + + java { + withJavadocJar() + withSourcesJar() + } + + // conditionals for publications + tasks.withType(PublishToMavenRepository).configureEach { + onlyIf { + (repository == publishing.repositories.nexusRepository && + project.hasProperty('nexus_user') && + project.hasProperty('nexus_password') && + project.hasProperty('nexus_url')) || + (repository == publishing.repositories.sonatype && + project.hasProperty('sonatypeUsername') && + project.hasProperty('sonatypePassword')) || + (repository == publishing.repositories.localRepo) + } + } + tasks.withType(Sign).configureEach { + onlyIf { + project.hasProperty('signingKey') && + project.hasProperty('signingPassword') + } + } + // disable running task 'initializeSonatypeStagingRepository' on a gitlab + tasks.configureEach { task -> + if (task.name == 'initializeSonatypeStagingRepository' && + !(project.hasProperty('sonatypeUsername') && project.hasProperty('sonatypePassword')) + ) { + task.enabled = false + } + } + + publishing { + publications { + mavenJava(MavenPublication) { + from(components.java) + pom { + name = project.name + packaging = 'jar' + description = rootProject.description + url = vcs_url + scm { + url = vcs_url + } + licenses { + license { + name = 'The Apache License, Version 2.0' + url = 'http://www.apache.org/licenses/LICENSE-2.0.txt' + } + } + developers { + developer { + id = 'developer' + name = 'developer' + email = 'developer@exactpro.com' + } + } + scm { + url = vcs_url + } + } + } + } + repositories { + maven { + name = 'localRepo' + url = sharedDir + } + //Nexus repo to publish from gitlab + maven { + name = 'nexusRepository' + credentials { + username = project.findProperty('nexus_user') + password = project.findProperty('nexus_password') + } + url = project.findProperty('nexus_url') + } + } + } + + signing { + String signingKey = findProperty("signingKey") + String signingPassword = findProperty("signingPassword") + useInMemoryPgpKeys(signingKey, signingPassword) + sign publishing.publications.mavenJava + } } nexusPublishing { - repositories { - sonatype { - nexusUrl.set(uri("https://s01.oss.sonatype.org/service/local/")) - snapshotRepositoryUrl.set(uri("https://s01.oss.sonatype.org/content/repositories/snapshots/")) - } - } + repositories { + sonatype { + nexusUrl.set(uri("https://s01.oss.sonatype.org/service/local/")) + snapshotRepositoryUrl.set(uri("https://s01.oss.sonatype.org/content/repositories/snapshots/")) + } + } } - -tasks.create("listProjects") { - doLast { - subprojects.forEach { println(it.name) } - } +tasks.register("listProjects") { + doLast { + subprojects.forEach { println(it.name) } + } } clean { - delete sharedDir + delete sharedDir } + static def getDate() { - def date = new Date() - def formattedDate = date.format('yyyyMMdd-HH:mm:ss.SSS') - return formattedDate + def date = new Date() + def formattedDate = date.format('yyyyMMdd-HH:mm:ss.SSS') + return formattedDate } \ No newline at end of file diff --git a/cradle-cassandra/build.gradle b/cradle-cassandra/build.gradle index 57fef7769..c9d1feef1 100644 --- a/cradle-cassandra/build.gradle +++ b/cradle-cassandra/build.gradle @@ -1,48 +1,52 @@ ext { - driver_version = '4.15.0' + driver_version = '4.15.0' } -configurations.all { - exclude group: 'com.esri.geometry' - exclude group: 'org.apache.tinkerpop' +configurations.configureEach { + exclude group: 'com.esri.geometry' + exclude group: 'org.apache.tinkerpop' } dependencies { - api project(":cradle-core") - - implementation "org.apache.commons:commons-lang3" - implementation 'com.exactpro.th2:task-utils:0.1.2' - - implementation "com.datastax.oss:java-driver-core:${driver_version}" - implementation "com.datastax.oss:java-driver-query-builder:${driver_version}" - implementation "com.datastax.oss:java-driver-mapper-processor:${driver_version}" - implementation "com.datastax.oss:java-driver-mapper-runtime:${driver_version}" - implementation 'io.prometheus:simpleclient_dropwizard:0.16.0' - - // this section is required to bypass failing vulnerability check caused by cassandra driver's transitive dependencies - annotationProcessor platform('com.exactpro.th2:bom:4.3.0') - annotationProcessor "com.datastax.oss:java-driver-mapper-processor:${driver_version}" - // --- - - testImplementation 'org.testng:testng:7.1.0' - testImplementation 'org.assertj:assertj-core:3.12.2' - testImplementation "org.apache.logging.log4j:log4j-slf4j2-impl" - testImplementation 'org.apache.logging.log4j:log4j-core' + api project(":cradle-core") + + implementation "org.apache.commons:commons-lang3" + implementation 'com.exactpro.th2:task-utils:0.1.2' + + implementation "com.datastax.oss:java-driver-core:${driver_version}" + implementation "com.datastax.oss:java-driver-query-builder:${driver_version}" + implementation "com.datastax.oss:java-driver-mapper-processor:${driver_version}" + implementation "com.datastax.oss:java-driver-mapper-runtime:${driver_version}" + implementation 'io.prometheus:simpleclient_dropwizard:0.16.0' + + // this section is required to bypass failing vulnerability check caused by cassandra driver's transitive dependencies + annotationProcessor platform('com.exactpro.th2:bom:4.3.0') + annotationProcessor "com.datastax.oss:java-driver-mapper-processor:${driver_version}" + // --- + + testImplementation 'org.testng:testng:7.7.0' + testImplementation 'org.assertj:assertj-core:3.12.2' + testImplementation "org.apache.logging.log4j:log4j-slf4j2-impl" + testImplementation 'org.apache.logging.log4j:log4j-core' } def gen_dir = 'build/generated/sources/annotationProcessor/main' sourceSets.main.java.srcDirs += [gen_dir] compileJava { - options.annotationProcessorGeneratedSourcesDirectory = file("${projectDir}/${gen_dir}") + options.annotationProcessorGeneratedSourcesDirectory = file("${projectDir}/${gen_dir}") } test { - useTestNG() + useTestNG() } jar { - manifest { - attributes("Specification-Title": "Cradle Cassandra") - } + manifest { + attributes("Specification-Title": "Cradle Cassandra") + } +} + +dependencyCheck { + suppressionFile = "${rootDir}/suppressions.xml" } \ No newline at end of file From b16f7f75723f4989641f29635a562293bfa9d825 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Mon, 19 Jun 2023 10:27:45 +0400 Subject: [PATCH 26/31] Refactored build.gradle --- build.gradle | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/build.gradle b/build.gradle index 00f858c07..d36d5e6b0 100644 --- a/build.gradle +++ b/build.gradle @@ -83,8 +83,7 @@ subprojects { project.hasProperty('nexus_url')) || (repository == publishing.repositories.sonatype && project.hasProperty('sonatypeUsername') && - project.hasProperty('sonatypePassword')) || - (repository == publishing.repositories.localRepo) + project.hasProperty('sonatypePassword')) } } tasks.withType(Sign).configureEach { @@ -134,10 +133,6 @@ subprojects { } } repositories { - maven { - name = 'localRepo' - url = sharedDir - } //Nexus repo to publish from gitlab maven { name = 'nexusRepository' @@ -172,10 +167,6 @@ tasks.register("listProjects") { } } -clean { - delete sharedDir -} - static def getDate() { def date = new Date() def formattedDate = date.format('yyyyMMdd-HH:mm:ss.SSS') From 8a1a37d9eb241f52aee34b876e3824d45a58ac06 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 21 Jun 2023 17:02:01 +0400 Subject: [PATCH 27/31] Corrected configuration for vulnerability check --- build.gradle | 1 + cradle-core/build.gradle | 32 ++++++++++++++++++-------------- 2 files changed, 19 insertions(+), 14 deletions(-) diff --git a/build.gradle b/build.gradle index d36d5e6b0..3e6072d0c 100644 --- a/build.gradle +++ b/build.gradle @@ -15,6 +15,7 @@ allprojects { dependencyCheck { formats = ['SARIF', 'JSON', 'HTML'] failBuildOnCVSS = 5 + suppressionFile = "${rootDir}/suppressions.xml" analyzers { assemblyEnabled = false nugetconfEnabled = false diff --git a/cradle-core/build.gradle b/cradle-core/build.gradle index 7abd0ec5c..cc40501da 100644 --- a/cradle-core/build.gradle +++ b/cradle-core/build.gradle @@ -1,25 +1,29 @@ dependencies { - api platform('com.exactpro.th2:bom:4.3.0') + api platform('com.exactpro.th2:bom:4.3.0') - implementation "com.fasterxml.jackson.core:jackson-databind" - implementation "org.apache.commons:commons-lang3" + implementation "com.fasterxml.jackson.core:jackson-databind" + implementation "org.apache.commons:commons-lang3" - implementation "org.slf4j:slf4j-api" - implementation 'com.google.guava:guava' - implementation 'org.lz4:lz4-java:1.8.0' + implementation "org.slf4j:slf4j-api" + implementation 'com.google.guava:guava' + implementation 'org.lz4:lz4-java:1.8.0' - testImplementation 'org.apache.logging.log4j:log4j-slf4j2-impl' - testImplementation 'org.apache.logging.log4j:log4j-core' - testImplementation 'org.testng:testng:7.1.0' - testImplementation 'org.assertj:assertj-core:3.12.2' + testImplementation 'org.apache.logging.log4j:log4j-slf4j2-impl' + testImplementation 'org.apache.logging.log4j:log4j-core' + testImplementation 'org.testng:testng:7.7.0' + testImplementation 'org.assertj:assertj-core:3.12.2' } test { - useTestNG() + useTestNG() } jar { - manifest { - attributes("Specification-Title": "Cradle Core") - } + manifest { + attributes("Specification-Title": "Cradle Core") + } +} + +dependencyCheck { + suppressionFile = "${rootDir}/suppressions.xml" } \ No newline at end of file From 9c98449975010ea711ebbb92eda79dceda1d5577 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Wed, 12 Jul 2023 16:41:56 +0400 Subject: [PATCH 28/31] [TH2-4887] Used bom:4.4.0 --- cradle-cassandra/build.gradle | 2 +- cradle-core/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/cradle-cassandra/build.gradle b/cradle-cassandra/build.gradle index c9d1feef1..cf2d111f7 100644 --- a/cradle-cassandra/build.gradle +++ b/cradle-cassandra/build.gradle @@ -20,7 +20,7 @@ dependencies { implementation 'io.prometheus:simpleclient_dropwizard:0.16.0' // this section is required to bypass failing vulnerability check caused by cassandra driver's transitive dependencies - annotationProcessor platform('com.exactpro.th2:bom:4.3.0') + annotationProcessor platform('com.exactpro.th2:bom:4.4.0') annotationProcessor "com.datastax.oss:java-driver-mapper-processor:${driver_version}" // --- diff --git a/cradle-core/build.gradle b/cradle-core/build.gradle index cc40501da..190437e62 100644 --- a/cradle-core/build.gradle +++ b/cradle-core/build.gradle @@ -1,5 +1,5 @@ dependencies { - api platform('com.exactpro.th2:bom:4.3.0') + api platform('com.exactpro.th2:bom:4.4.0') implementation "com.fasterxml.jackson.core:jackson-databind" implementation "org.apache.commons:commons-lang3" From bdc95311eeb37ca9b0e9d9c2bc682b5227e0c898 Mon Sep 17 00:00:00 2001 From: "nikita.smirnov" Date: Fri, 14 Jul 2023 12:14:55 +0400 Subject: [PATCH 29/31] Reformat build.gradle --- cradle-cassandra/build.gradle | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/cradle-cassandra/build.gradle b/cradle-cassandra/build.gradle index c4500e1d0..e9bbfc0e5 100644 --- a/cradle-cassandra/build.gradle +++ b/cradle-cassandra/build.gradle @@ -18,7 +18,7 @@ dependencies { implementation "com.datastax.oss:java-driver-mapper-processor:${driver_version}" implementation "com.datastax.oss:java-driver-mapper-runtime:${driver_version}" implementation 'io.prometheus:simpleclient_dropwizard:0.16.0' - implementation 'com.google.guava:guava' + implementation 'com.google.guava:guava' // this section is required to bypass failing vulnerability check caused by cassandra driver's transitive dependencies annotationProcessor platform('com.exactpro.th2:bom:4.4.0') @@ -29,8 +29,8 @@ dependencies { testImplementation 'org.assertj:assertj-core:3.12.2' testImplementation "org.apache.logging.log4j:log4j-slf4j2-impl" testImplementation 'org.apache.logging.log4j:log4j-core' - testImplementation 'org.mockito:mockito-core:5.2.0' - testImplementation 'org.testcontainers:cassandra:1.17.5' + testImplementation 'org.mockito:mockito-core:5.2.0' + testImplementation 'org.testcontainers:cassandra:1.17.5' } def gen_dir = 'build/generated/sources/annotationProcessor/main' @@ -42,15 +42,15 @@ compileJava { test { useTestNG() { - suites "src/test/resources/core.xml" - } + suites "src/test/resources/core.xml" + } } tasks.register('integrationTest', Test) { - group = 'verification' - useTestNG() { - suites "src/test/resources/integration.xml" - } + group = 'verification' + useTestNG() { + suites "src/test/resources/integration.xml" + } } jar { From fe6f79ee018735c4774c71ed11641ba12213304e Mon Sep 17 00:00:00 2001 From: Vakhtang Donadze <51744864+vdonadze@users.noreply.github.com> Date: Mon, 24 Jul 2023 15:53:28 +0000 Subject: [PATCH 30/31] [TH2-4998] Find nearest batch start time if filter start time is after batch start time (#246) --- .../AbstractMessageIteratorProvider.java | 2 + .../messages/GroupedMessageBatchOperator.java | 28 ++- .../GroupedMessageIteratorProvider.java | 92 +++++++++- ...nsApiTest.java => BaseMessageApiTest.java} | 52 ++++-- .../GetGroupedMessageBatchesApiTest.java | 160 ++++++++++++++++++ .../messages/GetMessagesApiTest.java | 122 +++++++++++++ .../messages/PageGroupsApiTest.java | 2 +- ...nsApiTest.java => PageMessageApiTest.java} | 2 +- 8 files changed, 432 insertions(+), 28 deletions(-) rename cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/{BaseSessionsApiTest.java => BaseMessageApiTest.java} (69%) create mode 100644 cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetGroupedMessageBatchesApiTest.java create mode 100644 cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetMessagesApiTest.java rename cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/{PageSessionsApiTest.java => PageMessageApiTest.java} (98%) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/AbstractMessageIteratorProvider.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/AbstractMessageIteratorProvider.java index 20b985865..aac8d6552 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/AbstractMessageIteratorProvider.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/AbstractMessageIteratorProvider.java @@ -116,6 +116,7 @@ public AbstractMessageIteratorProvider(String requestInfo, MessageFilter filter, this.iterationCondition = iterationCondition; } + //TODO refactor this method to assigns firstPage outside the method protected FilterForGreater createLeftBoundFilter(MessageFilter filter) throws CradleStorageException { FilterForGreater result = filter.getTimestampFrom(); @@ -173,6 +174,7 @@ private LocalTime getNearestBatchTime(PageInfo page, String sessionAlias, String return null; } + //TODO refactor this method to assign last page outside of this method. protected FilterForLess createRightBoundFilter(MessageFilter filter) { FilterForLess result = filter.getTimestampTo(); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchOperator.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchOperator.java index c1d79176f..cd80637f4 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchOperator.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchOperator.java @@ -19,22 +19,48 @@ import com.datastax.oss.driver.api.core.MappedAsyncPagingIterable; import com.datastax.oss.driver.api.core.cql.AsyncResultSet; import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; +import com.datastax.oss.driver.api.core.cql.Row; import com.datastax.oss.driver.api.mapper.annotations.Dao; import com.datastax.oss.driver.api.mapper.annotations.Delete; +import com.datastax.oss.driver.api.mapper.annotations.Query; import com.datastax.oss.driver.api.mapper.annotations.QueryProvider; import com.exactpro.cradle.cassandra.dao.CommonQueryProvider; import com.exactpro.cradle.cassandra.retries.SelectQueryExecutor; +import java.time.LocalDate; +import java.time.LocalTime; import java.util.concurrent.CompletableFuture; import java.util.function.Function; +import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.FIELD_FIRST_MESSAGE_TIME; +import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.FIELD_BOOK; +import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.FIELD_PAGE; +import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.FIELD_ALIAS_GROUP; +import static com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchEntity.FIELD_FIRST_MESSAGE_DATE; + + @Dao public interface GroupedMessageBatchOperator { @QueryProvider(providerClass = CommonQueryProvider.class, entityHelpers = GroupedMessageBatchEntity.class) CompletableFuture> getByFilter(CassandraGroupedMessageFilter filter, SelectQueryExecutor selectExecutor, String queryInfo, Function attributes); - + + @Query( "SELECT " + + FIELD_FIRST_MESSAGE_TIME + " " + + "FROM ${qualifiedTableId} " + + "WHERE " + + FIELD_BOOK + "=:book AND " + + FIELD_PAGE + "=:page AND " + + FIELD_ALIAS_GROUP + " =:groupAlias AND " + + "(" + FIELD_FIRST_MESSAGE_DATE + ", " + FIELD_FIRST_MESSAGE_TIME + ") <= (:messageDate, :messageTime) " + + "ORDER BY " + + FIELD_FIRST_MESSAGE_DATE + " DESC, " + + FIELD_FIRST_MESSAGE_TIME + " DESC LIMIT 1") + CompletableFuture getNearestTime(String book, String page, String groupAlias, + LocalDate messageDate, LocalTime messageTime, + Function attributes); + @QueryProvider(providerClass = GroupedMessageBatchInserter.class, entityHelpers = GroupedMessageBatchEntity.class, providerMethod = "insert") CompletableFuture write(GroupedMessageBatchEntity message, Function attributes); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java index 94eb649d3..8af0a1f8a 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageIteratorProvider.java @@ -17,6 +17,7 @@ package com.exactpro.cradle.cassandra.dao.messages; import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; +import com.datastax.oss.driver.api.core.cql.Row; import com.exactpro.cradle.BookInfo; import com.exactpro.cradle.Order; import com.exactpro.cradle.PageId; @@ -28,20 +29,28 @@ import com.exactpro.cradle.cassandra.retries.SelectQueryExecutor; import com.exactpro.cradle.cassandra.utils.FilterUtils; import com.exactpro.cradle.cassandra.workers.MessagesWorker; +import com.exactpro.cradle.filters.FilterForGreater; +import com.exactpro.cradle.filters.FilterForLess; import com.exactpro.cradle.iterators.ConvertingIterator; import com.exactpro.cradle.messages.GroupedMessageFilter; import com.exactpro.cradle.messages.StoredGroupedMessageBatch; import com.exactpro.cradle.utils.CradleStorageException; +import com.exactpro.cradle.utils.TimeUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; import java.util.Iterator; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; +import static com.exactpro.cradle.cassandra.dao.messages.MessageBatchEntity.FIELD_FIRST_MESSAGE_TIME; + public class GroupedMessageIteratorProvider extends IteratorProvider { public static final Logger logger = LoggerFactory.getLogger(GroupedMessageIteratorProvider.class); @@ -52,8 +61,9 @@ public class GroupedMessageIteratorProvider extends IteratorProvider leftBoundFilter; + protected final FilterForLess rightBoundFilter; + protected PageInfo firstPage, lastPage; private final Function readAttrs; /** limit must be strictly positive ( limit greater than 0 ) */ private final int limit; @@ -80,9 +90,10 @@ public GroupedMessageIteratorProvider(String requestInfo, this.filter = filter; this.limit = filter.getLimit(); this.returned = new AtomicInteger(); - // TODO: Get message batch before *from* timestamp - this.firstPage = FilterUtils.findFirstPage(filter.getPageId(), filter.getFrom(), book); - this.lastPage = FilterUtils.findLastPage(filter.getPageId(), filter.getTo(), book); + this.leftBoundFilter = createLeftBoundFilter(filter); + this.firstPage = FilterUtils.findFirstPage(filter.getPageId(), leftBoundFilter, book); + this.rightBoundFilter = createRightBoundFilter(filter); + this.lastPage = FilterUtils.findLastPage(filter.getPageId(), rightBoundFilter, book); this.order = order; // Filter should be initialized last as it might use above initialized properties @@ -94,12 +105,79 @@ private CassandraGroupedMessageFilter createInitialFilter(GroupedMessageFilter f book.getId().getName(), getFirstPage().getId().getName(), filter.getGroupName(), - filter.getFrom(), - filter.getTo(), + leftBoundFilter, + rightBoundFilter, order, filter.getLimit()); } + //TODO refactor or split this method to avoid findFirstPage calculation multiple times. + private FilterForGreater createLeftBoundFilter(GroupedMessageFilter filter) throws CradleStorageException + { + FilterForGreater result = filter.getFrom(); + var firstPageLocal = FilterUtils.findFirstPage(filter.getPageId(), result, book); + Instant leftBoundFromPage = firstPageLocal.getStarted(); + if (result == null || (filter.getPageId() != null && leftBoundFromPage.isAfter(result.getValue()))) + return FilterForGreater.forGreaterOrEquals(leftBoundFromPage); + + // If the page wasn't specified in the filter, we should find a batch with a lower date, + // which may contain messages that satisfy the original condition + LocalDateTime leftBoundLocalDate = TimeUtils.toLocalTimestamp(result.getValue()); + LocalTime nearestBatchTime = getNearestBatchTime( + firstPageLocal, + filter.getGroupName(), + leftBoundLocalDate.toLocalDate(), + leftBoundLocalDate.toLocalTime()); + + if (nearestBatchTime != null) + { + Instant nearestBatchInstant = TimeUtils.toInstant(leftBoundLocalDate.toLocalDate(), nearestBatchTime); + if (nearestBatchInstant.isBefore(result.getValue())) + result = FilterForGreater.forGreaterOrEquals(nearestBatchInstant); + } + + return result; + } + + private LocalTime getNearestBatchTime(PageInfo page, String groupAlias, LocalDate messageDate, LocalTime messageTime) throws CradleStorageException + { + while (page != null) + { + CompletableFuture future = op.getNearestTime( + page.getId().getBookId().getName(), + page.getId().getName(), + groupAlias, + messageDate, + messageTime, + readAttrs); + try + { + Row row = future.get(); + if (row != null) + return row.getLocalTime(FIELD_FIRST_MESSAGE_TIME); + } + catch (Exception e) + { + throw new CradleStorageException("Error while getting left bound ", e); + } + if (TimeUtils.toLocalTimestamp(page.getStarted()).toLocalDate().isBefore(messageDate)) + return null; + page = book.getPreviousPage(page.getStarted()); + } + + return null; + } + + //TODO refactor or split this method to avoid findLastPage calculation multiple times. + protected FilterForLess createRightBoundFilter(GroupedMessageFilter filter) + { + FilterForLess result = filter.getTo(); + var lastPageLocal = FilterUtils.findLastPage(filter.getPageId(), result, book); + Instant endOfPage = lastPageLocal.getEnded() == null ? Instant.now() : lastPageLocal.getEnded(); + + return FilterForLess.forLessOrEquals(result == null || endOfPage.isBefore(result.getValue()) ? endOfPage : result.getValue()); + } + protected CassandraGroupedMessageFilter createNextFilter(CassandraGroupedMessageFilter prevFilter, int updatedLimit) { PageInfo prevPage = book.getPage(new PageId(book.getId(), prevFilter.getPage())); if (prevPage.equals(getLastPage())) diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseSessionsApiTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseMessageApiTest.java similarity index 69% rename from cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseSessionsApiTest.java rename to cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseMessageApiTest.java index 12c153312..ed8f1397c 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseSessionsApiTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/BaseMessageApiTest.java @@ -19,24 +19,17 @@ import com.exactpro.cradle.CoreStorageSettings; import com.exactpro.cradle.Direction; import com.exactpro.cradle.cassandra.integration.BaseCradleCassandraTest; -import com.exactpro.cradle.counters.Interval; import com.exactpro.cradle.messages.GroupedMessageBatchToStore; import com.exactpro.cradle.utils.CradleStorageException; -import org.assertj.core.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.Test; import java.io.IOException; -import java.time.temporal.ChronoUnit; import java.util.List; import java.util.Set; -import static org.assertj.core.util.Lists.newArrayList; - -public class BaseSessionsApiTest extends BaseCradleCassandraTest { - private static final Logger logger = LoggerFactory.getLogger(BaseSessionsApiTest.class); +public class BaseMessageApiTest extends BaseCradleCassandraTest { + private static final Logger logger = LoggerFactory.getLogger(BaseMessageApiTest.class); protected static final String GROUP1_NAME = "test_group1"; @@ -80,30 +73,53 @@ public class BaseSessionsApiTest extends BaseCradleCassandraTest { protected void generateData() { try { GroupedMessageBatchToStore b1 = new GroupedMessageBatchToStore(GROUP1_NAME, 1024, storeActionRejectionThreshold); - //page 0 + // page 0 contains 3 messages from batch 1 and 1 message from batch 2. + // contains 2 groups test_group1 and test_group2 + // contains 2 session aliases test_session_alias1 and test_session_alias2 b1.addMessage(generateMessage(SESSION_ALIAS1, Direction.FIRST, 2, 1L)); b1.addMessage(generateMessage(SESSION_ALIAS1, Direction.SECOND, 3, 2L)); b1.addMessage(generateMessage(SESSION_ALIAS2, Direction.SECOND, 6, 3L)); GroupedMessageBatchToStore b2 = new GroupedMessageBatchToStore(GROUP2_NAME, 1024, storeActionRejectionThreshold); b2.addMessage(generateMessage(SESSION_ALIAS2, Direction.FIRST, 5, 4L)); - //page 2 + + // page 1 contains 2 messages from batch 2. + // contains 1 group test_group2 + // contains 2 session aliases test_session_alias3 and test_session_alias4 b2.addMessage(generateMessage(SESSION_ALIAS3, Direction.SECOND, 18, 5L)); b2.addMessage(generateMessage(SESSION_ALIAS4, Direction.SECOND, 19, 6L)); - //page 3 + // page 2 contains 4 messages from batch 3. + // contains 1 group test_group3 + // contains 3 session aliases test_session_alias4, test_session_alias5 and test_session_alias6 GroupedMessageBatchToStore b3 = new GroupedMessageBatchToStore(GROUP3_NAME, 1024, storeActionRejectionThreshold); b3.addMessage(generateMessage(SESSION_ALIAS4, Direction.FIRST, 25, 7L)); b3.addMessage(generateMessage(SESSION_ALIAS5, Direction.SECOND, 26, 8L)); - b3.addMessage(generateMessage(SESSION_ALIAS5, Direction.SECOND, 27, 9L)); - b3.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 28, 10L)); + b3.addMessage(generateMessage(SESSION_ALIAS5, Direction.SECOND, 28, 9L)); + b3.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 29, 10L)); - //page 4 + // page 3 contains 2 messages from batch 4. + // contains 1 group test_group4 + // contains 1 session alias test_session_alias6 GroupedMessageBatchToStore b4 = new GroupedMessageBatchToStore(GROUP4_NAME, 1024, storeActionRejectionThreshold); b4.addMessage(generateMessage(SESSION_ALIAS6, Direction.FIRST, 35, 11L)); - b4.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 46, 12L)); - - List data = List.of(b1, b2, b3, b4); + b4.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 37, 12L)); + + // page 4 contains 1 message from batch 5. + // contains 1 group test_group3 + // contains 1 session alias test_session_alias6 + GroupedMessageBatchToStore b5 = new GroupedMessageBatchToStore(GROUP3_NAME, 1024, storeActionRejectionThreshold); + b5.addMessage(generateMessage(SESSION_ALIAS6, Direction.FIRST, 45, 13L)); + + // page 5 contains 1 message from batch 5 and 2 messages from batch 6. + // contains 1 group test_group3 + // contains 1 session alias test_session_alias6 + b5.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 52, 14L)); + + GroupedMessageBatchToStore b6 = new GroupedMessageBatchToStore(GROUP3_NAME, 1024, storeActionRejectionThreshold); + b6.addMessage(generateMessage(SESSION_ALIAS6, Direction.FIRST, 53, 15L)); + b6.addMessage(generateMessage(SESSION_ALIAS6, Direction.SECOND, 55, 16L)); + List data = List.of(b1, b2, b3, b4, b5, b6); for (GroupedMessageBatchToStore el : data) { storage.storeGroupedMessageBatch(el); diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetGroupedMessageBatchesApiTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetGroupedMessageBatchesApiTest.java new file mode 100644 index 000000000..5ce4d2bae --- /dev/null +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetGroupedMessageBatchesApiTest.java @@ -0,0 +1,160 @@ +/* + * Copyright 2023 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.cradle.cassandra.integration.messages; + +import com.exactpro.cradle.messages.GroupedMessageFilter; +import com.exactpro.cradle.utils.CradleStorageException; +import org.assertj.core.api.Assertions; +import org.assertj.core.util.Lists; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.time.temporal.ChronoUnit; + +public class GetGroupedMessageBatchesApiTest extends BaseMessageApiTest { + @BeforeClass + public void startUp() throws IOException, InterruptedException, CradleStorageException { + super.startUp(true); + generateData(); + } + + @Test(description = "Get grouped messages within 1 page. Interval start time is less than batch start time and end time more then batch end time") + public void getGroupedMessagesWithWideIntervalTest() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(24, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(1); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + + @Test(description = "Get grouped messages within 1 page. Interval start time is equal to batch end time.") + public void getGroupedMessagesWithIntervalAtEnd() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(29, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(1); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + @Test(description = "Get grouped messages withing interval that covers multiple pages. First batch start time is equal to interval start and interval end time is after last batch end.") + public void getGroupedMessagesWithWideIntervalTest2() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(25, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(58, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(4); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + Assertions.assertThat(resultAsList.get(1).getMessages().size()).isEqualTo(1); + Assertions.assertThat(resultAsList.get(2).getMessages().size()).isEqualTo(1); + Assertions.assertThat(resultAsList.get(3).getMessages().size()).isEqualTo(2); + } + + + @Test(description = "Get grouped messages withing interval that covers multiple pages. Interval start time is in the middle of the first batch and interval end is before the end of the last batch.") + public void getGroupedMessagesWithWideIntervalTest4() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(27, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(46, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(2); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + Assertions.assertThat(resultAsList.get(1).getMessages().size()).isEqualTo(1); + } + + @Test(description = "Get grouped messages withing interval with start time less than batch start time and end time less then batch end time") + public void getGroupedMessagesWithHalfWideIntervalTest0() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(24, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(28, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + + @Test(description = "Get grouped messages withing interval with start time less than batch start time and end time less then batch end time") + public void getGroupedMessagesWithHalfWideIntervalTest1() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(24, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(29, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + + @Test(description = "Get grouped messages withing interval with start time more than batch start time and end time more then batch end time") + public void getGroupedMessagesWithHalfWideIntervalTest2() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + + @Test(description = "Get grouped messages withing interval with start time more than batch start time and end time less then batch end time") + public void getGroupedMessagesWithNarrowIntervalTest3() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(28, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } + + @Test(description = "Get grouped messages withing interval with start time more than batch start time and end time less then batch end time") + public void getGroupedMessagesWithNarrowIntervalTest() throws CradleStorageException, IOException { + GroupedMessageFilter filter = GroupedMessageFilter.builder() + .groupName(GROUP3_NAME) + .bookId(bookId) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(29, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getGroupedMessageBatches(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.get(0).getMessages().size()).isEqualTo(4); + } +} diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetMessagesApiTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetMessagesApiTest.java new file mode 100644 index 000000000..8ed08b211 --- /dev/null +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/GetMessagesApiTest.java @@ -0,0 +1,122 @@ +/* + * Copyright 2023 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.cradle.cassandra.integration.messages; + +import com.exactpro.cradle.Direction; +import com.exactpro.cradle.messages.MessageFilter; +import com.exactpro.cradle.messages.MessageFilterBuilder; +import com.exactpro.cradle.utils.CradleStorageException; +import org.assertj.core.api.Assertions; +import org.assertj.core.util.Lists; +import org.testng.annotations.BeforeClass; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.time.temporal.ChronoUnit; + +public class GetMessagesApiTest extends BaseMessageApiTest { + @BeforeClass + public void startUp() throws IOException, InterruptedException, CradleStorageException { + super.startUp(true); + generateData(); + } + + @Test(description = "Get messages withing interval with start time less than batch start time and end time more then batch end time") + public void getMessagesWithWideIntervalTest() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(24, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(2); + } + + @Test(description = "Get messages withing interval with start time more than batch start time and end time more then batch end time") + public void getMessagesWithHalfWideIntervalTest() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(2); + } + + + @Test(description = "Get messages withing interval with start time more than batch start time and end time more then batch end time") + public void getMessagesWithHalfWideIntervalTest2() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(27, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(30, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(1); + } + + @Test(description = "Get messages withing interval with start time more than batch start time and end time less then batch end time") + public void getMessagesWithNarrowIntervalTest() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(29, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(2); + } + + @Test(description = "Get messages withing interval with start time more than batch start time and end time less then batch end time") + public void getMessagesWithNarrowIntervalTest2() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(28, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(1); + } + + @Test(description = "Get messages withing interval with start time more than batch start time and end time less then batch end time") + public void getMessagesWithNarrowIntervalTest3() throws CradleStorageException, IOException { + MessageFilter filter = new MessageFilterBuilder() + .bookId(bookId) + .sessionAlias(SESSION_ALIAS5) + .direction(Direction.SECOND) + .timestampFrom().isGreaterThanOrEqualTo(dataStart.plus(26, ChronoUnit.MINUTES)) + .timestampTo().isLessThan(dataStart.plus(27, ChronoUnit.MINUTES)) + .build(); + var actual = storage.getMessages(filter); + var resultAsList = Lists.newArrayList(actual.asIterable()); + Assertions.assertThat(resultAsList.size()).isEqualTo(1); + } +} diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageGroupsApiTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageGroupsApiTest.java index 1bff5a7e3..4d0af9fd0 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageGroupsApiTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageGroupsApiTest.java @@ -28,7 +28,7 @@ import static org.assertj.core.util.Lists.newArrayList; -public class PageGroupsApiTest extends BaseSessionsApiTest { +public class PageGroupsApiTest extends BaseMessageApiTest { @BeforeClass public void startUp() throws IOException, InterruptedException, CradleStorageException { super.startUp(true); diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageSessionsApiTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageMessageApiTest.java similarity index 98% rename from cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageSessionsApiTest.java rename to cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageMessageApiTest.java index 9d14dc9a7..a0b867e1a 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageSessionsApiTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/integration/messages/PageMessageApiTest.java @@ -28,7 +28,7 @@ import static org.assertj.core.util.Lists.newArrayList; -public class PageSessionsApiTest extends BaseSessionsApiTest { +public class PageMessageApiTest extends BaseMessageApiTest { @BeforeClass public void startUp() throws IOException, InterruptedException, CradleStorageException { From 042f0e7821207136c7e47a790a78f439a66bfcad Mon Sep 17 00:00:00 2001 From: Nikita Smirnov <46124551+Nikita-Smirnov-Exactpro@users.noreply.github.com> Date: Fri, 11 Aug 2023 06:43:42 +0000 Subject: [PATCH 31/31] =?UTF-8?q?[TH2-5004]=20save=20sessions=20and=20page?= =?UTF-8?q?=5Fsessions=20when=20storeIndividualMessag=E2=80=A6=20(#247)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * [TH2-5004] save sessions and page_sessions when storeIndividualMessageSessions is false * [TH2-5004] store statistics for sessions when store (#248) * [TH2-5004] refactored after review --- .../cassandra/CassandraCradleStorage.java | 80 +++++-- .../cassandra/CassandraStorageSettings.java | 18 +- .../cassandra/ReadThroughBookCache.java | 35 ++-- .../counters/EntityStatisticsCollector.java | 4 +- .../counters/MessageStatisticsCollector.java | 6 +- .../counters/SessionStatisticsCollector.java | 19 +- .../cassandra/dao/SerializedEntity.java | 15 +- .../dao/messages/PageSessionEntity.java | 22 +- .../dao/testevents/TestEventEntityUtils.java | 46 ++-- .../PagesInIntervalIteratorProvider.java | 3 +- .../utils/GroupedMessageEntityUtils.java | 9 +- .../utils/MessageBatchEntityUtils.java | 7 +- .../cassandra/workers/MessagesWorker.java | 139 +++++++++++-- .../cassandra/workers/StatisticsWorker.java | 36 +++- .../GroupedMessageBatchEntityTest.java | 7 +- .../dao/messages/MessageBatchEntityTest.java | 3 +- .../dao/testevents/TestEventEntityTest.java | 3 +- .../java/com/exactpro/cradle/BookCache.java | 19 +- .../exactpro/cradle/CoreStorageSettings.java | 21 +- .../com/exactpro/cradle/CradleStorage.java | 21 +- .../exactpro/cradle/DummyCradleStorage.java | 196 ++++++++---------- .../java/com/exactpro/cradle/PageInfo.java | 14 +- .../com/exactpro/cradle/counters/Counter.java | 10 +- .../serialization/EventBatchSerializer.java | 31 +-- .../serialization/MessageSerializer.java | 28 ++- .../serialization/SerializedEntityData.java | 10 +- .../SerializedMessageMetadata.java | 40 ++++ .../exactpro/cradle/utils/MessageUtils.java | 11 +- .../exactpro/cradle/utils/TestEventUtils.java | 9 +- 29 files changed, 564 insertions(+), 298 deletions(-) create mode 100644 cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedMessageMetadata.java diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java index e6b7224f1..c4ea778d2 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraCradleStorage.java @@ -23,19 +23,52 @@ import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder; import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; import com.datastax.oss.driver.api.core.cql.ResultSet; -import com.exactpro.cradle.*; +import com.exactpro.cradle.BookCache; +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.BookInfo; +import com.exactpro.cradle.BookListEntry; +import com.exactpro.cradle.BookManager; +import com.exactpro.cradle.BookToAdd; +import com.exactpro.cradle.CradleStorage; +import com.exactpro.cradle.Direction; +import com.exactpro.cradle.EntityType; +import com.exactpro.cradle.FrameType; +import com.exactpro.cradle.PageId; +import com.exactpro.cradle.PageInfo; +import com.exactpro.cradle.SessionRecordType; import com.exactpro.cradle.cassandra.connection.CassandraConnection; import com.exactpro.cradle.cassandra.connection.CassandraConnectionSettings; import com.exactpro.cradle.cassandra.counters.FrameInterval; -import com.exactpro.cradle.cassandra.dao.*; -import com.exactpro.cradle.cassandra.dao.books.*; +import com.exactpro.cradle.cassandra.dao.CassandraDataMapper; +import com.exactpro.cradle.cassandra.dao.CassandraDataMapperBuilder; +import com.exactpro.cradle.cassandra.dao.CassandraOperators; +import com.exactpro.cradle.cassandra.dao.books.BookEntity; +import com.exactpro.cradle.cassandra.dao.books.PageEntity; +import com.exactpro.cradle.cassandra.dao.books.PageNameEntity; +import com.exactpro.cradle.cassandra.dao.books.PageNameOperator; +import com.exactpro.cradle.cassandra.dao.books.PageOperator; import com.exactpro.cradle.cassandra.dao.intervals.CassandraIntervalsWorker; -import com.exactpro.cradle.cassandra.dao.messages.*; +import com.exactpro.cradle.cassandra.dao.messages.GroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.GroupedMessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.GroupsOperator; +import com.exactpro.cradle.cassandra.dao.messages.MessageBatchOperator; +import com.exactpro.cradle.cassandra.dao.messages.PageGroupEntity; +import com.exactpro.cradle.cassandra.dao.messages.PageGroupsIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.PageGroupsOperator; +import com.exactpro.cradle.cassandra.dao.messages.PageSessionEntity; +import com.exactpro.cradle.cassandra.dao.messages.PageSessionsIteratorProvider; +import com.exactpro.cradle.cassandra.dao.messages.PageSessionsOperator; +import com.exactpro.cradle.cassandra.dao.messages.SessionEntity; +import com.exactpro.cradle.cassandra.dao.messages.SessionsOperator; import com.exactpro.cradle.cassandra.dao.statistics.EntityStatisticsIteratorProvider; import com.exactpro.cradle.cassandra.dao.statistics.MessageStatisticsIteratorProvider; import com.exactpro.cradle.cassandra.dao.statistics.MessageStatisticsOperator; import com.exactpro.cradle.cassandra.dao.statistics.SessionStatisticsOperator; -import com.exactpro.cradle.cassandra.dao.testevents.*; +import com.exactpro.cradle.cassandra.dao.testevents.PageScopeEntity; +import com.exactpro.cradle.cassandra.dao.testevents.PageScopesIteratorProvider; +import com.exactpro.cradle.cassandra.dao.testevents.PageScopesOperator; +import com.exactpro.cradle.cassandra.dao.testevents.ScopeEntity; +import com.exactpro.cradle.cassandra.dao.testevents.TestEventOperator; import com.exactpro.cradle.cassandra.iterators.PagedIterator; import com.exactpro.cradle.cassandra.keyspaces.CradleInfoKeyspaceCreator; import com.exactpro.cradle.cassandra.metrics.DriverMetrics; @@ -56,7 +89,14 @@ import com.exactpro.cradle.counters.Interval; import com.exactpro.cradle.intervals.IntervalsWorker; import com.exactpro.cradle.iterators.ConvertingIterator; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.GroupedMessageFilter; +import com.exactpro.cradle.messages.MessageBatchToStore; +import com.exactpro.cradle.messages.MessageFilter; +import com.exactpro.cradle.messages.StoredGroupedMessageBatch; +import com.exactpro.cradle.messages.StoredMessage; +import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.resultset.CradleResultSet; import com.exactpro.cradle.testevents.StoredTestEvent; import com.exactpro.cradle.testevents.StoredTestEventId; @@ -69,8 +109,17 @@ import org.slf4j.LoggerFactory; import java.io.IOException; -import java.time.*; -import java.util.*; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; @@ -313,8 +362,7 @@ protected void doStoreGroupedMessageBatch(GroupedMessageBatchToStore batch, Page try { - messagesWorker.storeGroupedMessageBatch(batch, pageId).get(); - + messagesWorker.storeGroupedMessageBatch(batch, pageId, !settings.isStoreIndividualMessageSessions()).get(); if (settings.isStoreIndividualMessageSessions()) { for (MessageBatchToStore b : batch.getSessionMessageBatches()) { messagesWorker.storeMessageBatch(b, pageId).get(); @@ -344,8 +392,7 @@ protected CompletableFuture doStoreGroupedMessageBatchAsync(GroupedMessage { PageId pageId = page.getId(); - CompletableFuture future = messagesWorker.storeGroupedMessageBatch(batch, pageId); - + CompletableFuture future = messagesWorker.storeGroupedMessageBatch(batch, pageId, !settings.isStoreIndividualMessageSessions()); if (settings.isStoreIndividualMessageSessions()) { for (MessageBatchToStore b : batch.getSessionMessageBatches()) { future = future.thenComposeAsync((unused) -> messagesWorker.storeMessageBatch(b, pageId), composingService) @@ -889,17 +936,14 @@ protected Counter doGetCount(BookId bookId, EntityType entityType, Interval inte entityType.name(), interval.getStart().toString(), interval.getEnd().toString()); - try - { + try { return doGetCountAsync(bookId, entityType, interval).get(); - } - catch (Exception e) - { + } catch (Exception e) { throw new IOException("Error while getting " + queryInfo, e); } } - private CompletableFuture> doGetSessionsAsync (BookId bookId, Interval interval, SessionRecordType recordType) throws CradleStorageException { + private CompletableFuture> doGetSessionsAsync(BookId bookId, Interval interval, SessionRecordType recordType) throws CradleStorageException { String queryInfo = String.format("%s Aliases in book %s from %s to %s", recordType.name(), bookId.getName(), diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java index 80f26d687..13c6190ff 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/CassandraStorageSettings.java @@ -49,7 +49,6 @@ public class CassandraStorageSettings extends CoreStorageSettings { public static final int DEFAULT_COUNTER_PERSISTENCE_INTERVAL_MS = 1000; public static final long DEFAULT_EVENT_BATCH_DURATION_MILLIS = 5_000; public static final long DEFAULT_TIMEOUT = 5000; - public static final boolean DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS = true; public static final CompressionType DEFAULT_COMPRESSION_TYPE = CompressionType.ZLIB; @JsonIgnore @@ -85,7 +84,6 @@ public class CassandraStorageSettings extends CoreStorageSettings { private long eventBatchDurationMillis = DEFAULT_EVENT_BATCH_DURATION_MILLIS; - private boolean storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; private CompressionType compressionType = DEFAULT_COMPRESSION_TYPE; public CassandraStorageSettings() { @@ -138,11 +136,11 @@ public CassandraStorageSettings(CassandraStorageSettings settings) { this.sessionStatisticsCacheSize = settings.getSessionStatisticsCacheSize(); this.counterPersistenceInterval = settings.getCounterPersistenceInterval(); this.composingServiceThreads = settings.getComposingServiceThreads(); - this.bookRefreshIntervalMillis = settings.getBookRefreshIntervalMillis(); + setBookRefreshIntervalMillis(settings.getBookRefreshIntervalMillis()); this.eventBatchDurationMillis = settings.getEventBatchDurationMillis(); this.eventBatchDurationCacheSize = settings.getEventBatchDurationCacheSize(); - this.storeIndividualMessageSessions = settings.isStoreIndividualMessageSessions(); + setStoreIndividualMessageSessions(settings.isStoreIndividualMessageSessions()); this.compressionType = settings.getCompressionType(); } @@ -365,14 +363,6 @@ public void setEventBatchDurationCacheSize(int eventBatchDurationCacheSize) { this.eventBatchDurationCacheSize = eventBatchDurationCacheSize; } - public boolean isStoreIndividualMessageSessions() { - return storeIndividualMessageSessions; - } - - public void setStoreIndividualMessageSessions(boolean storeIndividualMessageSessions) { - this.storeIndividualMessageSessions = storeIndividualMessageSessions; - } - public CompressionType getCompressionType() { return compressionType; } @@ -409,9 +399,9 @@ public String toString() { ", composingServiceThreads=" + composingServiceThreads + ", multiRowResultExecutionPolicy=" + multiRowResultExecutionPolicy + ", singleRowResultExecutionPolicy=" + singleRowResultExecutionPolicy + - ", bookRefreshIntervalMillis=" + bookRefreshIntervalMillis + + ", bookRefreshIntervalMillis=" + getBookRefreshIntervalMillis() + ", eventBatchDurationMillis=" + eventBatchDurationMillis + - ", storeIndividualMessageSessions=" + storeIndividualMessageSessions + + ", storeIndividualMessageSessions=" + isStoreIndividualMessageSessions() + ", compressionType=" + compressionType + '}'; } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/ReadThroughBookCache.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/ReadThroughBookCache.java index 1e399c04a..5b8bf8377 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/ReadThroughBookCache.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/ReadThroughBookCache.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -16,15 +16,17 @@ package com.exactpro.cradle.cassandra; import com.datastax.oss.driver.api.core.cql.BoundStatementBuilder; -import com.exactpro.cradle.*; +import com.exactpro.cradle.BookCache; +import com.exactpro.cradle.BookId; +import com.exactpro.cradle.BookInfo; +import com.exactpro.cradle.PageInfo; import com.exactpro.cradle.cassandra.dao.CassandraOperators; import com.exactpro.cradle.cassandra.dao.books.BookEntity; import com.exactpro.cradle.cassandra.dao.books.PageEntity; import com.exactpro.cradle.errors.BookNotFoundException; import com.exactpro.cradle.utils.CradleStorageException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import java.time.LocalDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -33,9 +35,6 @@ import java.util.function.Function; public class ReadThroughBookCache implements BookCache { - private static final Logger logger = LoggerFactory.getLogger(ReadThroughBookCache.class); - - private final String UNSUPPORTED_SCHEMA_VERSION_FORMAT = "Unsupported schema version for the book \"%s\". Expected: %s, found: %s"; private final CassandraOperators operators; private final Map books; @@ -70,11 +69,21 @@ public boolean checkBook(BookId bookId) { return books.containsKey(bookId); } - public Collection loadPageInfo(BookId bookId, boolean loadRemoved) throws CradleStorageException - { + public Collection loadPageInfo(BookId bookId, boolean loadRemoved) { + Collection result = new ArrayList<>(); + for (PageEntity pageEntity : operators.getPageOperator().getAll(bookId.getName(), readAttrs)) { + if (loadRemoved || pageEntity.getRemoved() == null) { + result.add(pageEntity.toPageInfo()); + } + } + return result; + } + + @Override + public Collection loadPageInfo(BookId bookId, LocalDateTime start, boolean loadRemoved) { Collection result = new ArrayList<>(); - for (PageEntity pageEntity : operators.getPageOperator().getAll(bookId.getName(), readAttrs)) - { + for (PageEntity pageEntity : operators.getPageOperator().get( + bookId.getName(), start.toLocalDate(), start.toLocalTime(), readAttrs)) { if (loadRemoved || pageEntity.getRemoved() == null) { result.add(pageEntity.toPageInfo()); } @@ -82,7 +91,7 @@ public Collection loadPageInfo(BookId bookId, boolean loadRemoved) thr return result; } - public BookInfo loadBook (BookId bookId) throws CradleStorageException { + public BookInfo loadBook(BookId bookId) throws CradleStorageException { BookEntity bookEntity = operators.getBookOperator().get(bookId.getName(), readAttrs); if (bookEntity == null) { @@ -90,7 +99,7 @@ public BookInfo loadBook (BookId bookId) throws CradleStorageException { } if (!bookEntity.getSchemaVersion().equals(schemaVersion)) { - throw new CradleStorageException(String.format(UNSUPPORTED_SCHEMA_VERSION_FORMAT, + throw new CradleStorageException(String.format("Unsupported schema version for the book \"%s\". Expected: %s, found: %s", bookEntity.getName(), schemaVersion, bookEntity.getSchemaVersion())); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/EntityStatisticsCollector.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/EntityStatisticsCollector.java index aad5c38c0..283bfd68e 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/EntityStatisticsCollector.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/EntityStatisticsCollector.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -21,5 +21,5 @@ import java.util.Collection; public interface EntityStatisticsCollector { - void updateEntityBatchStatistics(BookId bookId, BookStatisticsRecordsCaches.EntityKey entityKey, Collection batchMetadata); + void updateEntityBatchStatistics(BookId bookId, BookStatisticsRecordsCaches.EntityKey entityKey, Collection batchMetadata); } \ No newline at end of file diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/MessageStatisticsCollector.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/MessageStatisticsCollector.java index c299a90a9..7dbb432cf 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/MessageStatisticsCollector.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/MessageStatisticsCollector.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -16,10 +16,10 @@ package com.exactpro.cradle.cassandra.counters; import com.exactpro.cradle.BookId; -import com.exactpro.cradle.serialization.SerializedEntityMetadata; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import java.util.Collection; public interface MessageStatisticsCollector { - void updateMessageBatchStatistics(BookId bookId, String page, String sessionAlias, String direction, Collection batchMetadata); + void updateMessageBatchStatistics(BookId bookId, String page, String sessionAlias, String direction, Collection batchMetadata); } \ No newline at end of file diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionStatisticsCollector.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionStatisticsCollector.java index 91fc4bea7..844d02006 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionStatisticsCollector.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/counters/SessionStatisticsCollector.java @@ -1,11 +1,26 @@ +/* + * Copyright 2022-2023 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.cradle.cassandra.counters; import com.exactpro.cradle.BookId; import com.exactpro.cradle.SessionRecordType; -import com.exactpro.cradle.serialization.SerializedEntityMetadata; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import java.util.Collection; public interface SessionStatisticsCollector { - void updateSessionStatistics(BookId bookId, String page, SessionRecordType recordType, String session, Collection batchMetadata); + void updateSessionStatistics(BookId bookId, String page, SessionRecordType recordType, String session, Collection batchMetadata); } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/SerializedEntity.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/SerializedEntity.java index 3ddd9ef09..030172e4e 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/SerializedEntity.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/SerializedEntity.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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,22 +17,23 @@ package com.exactpro.cradle.cassandra.dao; import com.exactpro.cradle.serialization.SerializedEntityData; +import com.exactpro.cradle.serialization.SerializedEntityMetadata; -public class SerializedEntity{ +public class SerializedEntity { - private final SerializedEntityData serializedEntityData; - private final T entity; + private final SerializedEntityData serializedEntityData; + private final E entity; - public SerializedEntity(SerializedEntityData serializedEntityData, T entity) { + public SerializedEntity(SerializedEntityData serializedEntityData, E entity) { this.serializedEntityData = serializedEntityData; this.entity = entity; } - public SerializedEntityData getSerializedEntityData() { + public SerializedEntityData getSerializedEntityData() { return serializedEntityData; } - public T getEntity() { + public E getEntity() { return entity; } } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/PageSessionEntity.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/PageSessionEntity.java index 26ede9183..8478bb76a 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/PageSessionEntity.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/messages/PageSessionEntity.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -16,7 +16,11 @@ package com.exactpro.cradle.cassandra.dao.messages; -import com.datastax.oss.driver.api.mapper.annotations.*; +import com.datastax.oss.driver.api.mapper.annotations.ClusteringColumn; +import com.datastax.oss.driver.api.mapper.annotations.CqlName; +import com.datastax.oss.driver.api.mapper.annotations.Entity; +import com.datastax.oss.driver.api.mapper.annotations.PartitionKey; +import com.datastax.oss.driver.api.mapper.annotations.PropertyStrategy; import com.exactpro.cradle.PageId; import com.exactpro.cradle.messages.StoredMessageId; @@ -33,6 +37,7 @@ public class PageSessionEntity { public static final String FIELD_SESSION_ALIAS = "session_alias"; public static final String FIELD_DIRECTION = "direction"; + @SuppressWarnings("DefaultAnnotationParam") @PartitionKey(0) @CqlName(FIELD_BOOK) private final String book; @@ -41,6 +46,7 @@ public class PageSessionEntity { @CqlName(FIELD_PAGE) private final String page; + @SuppressWarnings("DefaultAnnotationParam") @ClusteringColumn(0) @CqlName(FIELD_SESSION_ALIAS) private final String sessionAlias; @@ -57,21 +63,21 @@ public PageSessionEntity(String book, String page, String sessionAlias, String d } public PageSessionEntity(StoredMessageId messageId, PageId pageId) { - this.book = messageId.getBookId().getName(); - this.page = pageId.getName(); - this.sessionAlias = messageId.getSessionAlias(); - this.direction = messageId.getDirection().getLabel(); + this(messageId.getBookId().getName(), pageId.getName(), messageId.getSessionAlias(), messageId.getDirection().getLabel()); } - public String getBook() { + public String getBook() { return book; } + public String getPage() { return page; } - public String getSessionAlias() { + + public String getSessionAlias() { return sessionAlias; } + public String getDirection() { return direction; } diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityUtils.java index e9548fa0c..295b936e1 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityUtils.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -20,8 +20,19 @@ import com.exactpro.cradle.cassandra.dao.SerializedEntity; import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.serialization.SerializedEntityData; -import com.exactpro.cradle.testevents.*; -import com.exactpro.cradle.utils.*; +import com.exactpro.cradle.serialization.SerializedEntityMetadata; +import com.exactpro.cradle.testevents.BatchedStoredTestEvent; +import com.exactpro.cradle.testevents.StoredTestEvent; +import com.exactpro.cradle.testevents.StoredTestEventBatch; +import com.exactpro.cradle.testevents.StoredTestEventId; +import com.exactpro.cradle.testevents.StoredTestEventSingle; +import com.exactpro.cradle.testevents.TestEventToStore; +import com.exactpro.cradle.utils.CompressException; +import com.exactpro.cradle.utils.CompressionType; +import com.exactpro.cradle.utils.CradleIdException; +import com.exactpro.cradle.utils.CradleStorageException; +import com.exactpro.cradle.utils.TestEventUtils; +import com.exactpro.cradle.utils.TimeUtils; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +71,7 @@ private static StoredTestEventId createParentId(TestEventEntity testEventEntity) } - private static byte[] restoreContent(TestEventEntity testEventEntity, StoredTestEventId eventId) throws IOException, DataFormatException, CompressException { + private static byte[] restoreContent(TestEventEntity testEventEntity, StoredTestEventId eventId) throws CompressException { ByteBuffer content = testEventEntity.getContent(); if (content == null) return null; @@ -74,9 +85,8 @@ private static byte[] restoreContent(TestEventEntity testEventEntity, StoredTest } private static Set restoreMessages(TestEventEntity testEventEntity, BookId bookId) - throws IOException, DataFormatException, CradleIdException - { - ByteBuffer messages = testEventEntity.getMessages(); + throws IOException { + ByteBuffer messages = testEventEntity.getMessages(); if (messages == null) return null; @@ -85,8 +95,7 @@ private static Set restoreMessages(TestEventEntity testEventEnt } private static Map> restoreBatchMessages(TestEventEntity testEventEntity, BookId bookId) - throws IOException, DataFormatException, CradleIdException - { + throws IOException { ByteBuffer messages = testEventEntity.getMessages(); if (messages == null) return null; @@ -97,7 +106,7 @@ private static Map> restoreBatchMessages private static StoredTestEventSingle toStoredTestEventSingle(TestEventEntity testEventEntity, PageId pageId, StoredTestEventId eventId, byte[] content) - throws IOException, CradleStorageException, DataFormatException, CradleIdException + throws IOException, CradleIdException { Set messages = restoreMessages(testEventEntity, pageId.getBookId()); return new StoredTestEventSingle(eventId, testEventEntity.getName(), testEventEntity.getType(), createParentId(testEventEntity), @@ -105,7 +114,7 @@ private static StoredTestEventSingle toStoredTestEventSingle(TestEventEntity tes } private static StoredTestEventBatch toStoredTestEventBatch(TestEventEntity testEventEntity, PageId pageId, StoredTestEventId eventId, byte[] content) - throws IOException, CradleStorageException, DataFormatException, CradleIdException + throws IOException, CradleStorageException, CradleIdException { Collection children = TestEventUtils.deserializeTestEvents(content, eventId); Map> messages = restoreBatchMessages(testEventEntity, pageId.getBookId()); @@ -113,24 +122,23 @@ private static StoredTestEventBatch toStoredTestEventBatch(TestEventEntity testE children, messages, pageId, null, testEventEntity.getRecDate()); } - public static Instant getEndTimestamp (TestEventEntity entity) { + public static Instant getEndTimestamp(TestEventEntity entity) { return TimeUtils.toInstant(entity.getEndDate(), entity.getEndTime()); } - public static Instant getStartTimestamp(TestEventEntity entity) - { + public static Instant getStartTimestamp(TestEventEntity entity) { return TimeUtils.toInstant(entity.getStartDate(), entity.getStartTime()); } - public static SerializedEntity toSerializedEntity(TestEventToStore event, - PageId pageId, - CompressionType compressionType, - int maxUncompressedSize) throws IOException, CompressException { + public static SerializedEntity toSerializedEntity(TestEventToStore event, + PageId pageId, + CompressionType compressionType, + int maxUncompressedSize) throws IOException, CompressException { TestEventEntity.TestEventEntityBuilder builder = TestEventEntity.builder(); logger.debug("Creating entity from test event '{}'", event.getId()); - SerializedEntityData serializedEntityData = TestEventUtils.getTestEventContent(event); + SerializedEntityData serializedEntityData = TestEventUtils.getTestEventContent(event); byte[] content = serializedEntityData.getSerializedData(); boolean compressed = false; diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/resultset/PagesInIntervalIteratorProvider.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/resultset/PagesInIntervalIteratorProvider.java index 67ca217a0..f11f5b846 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/resultset/PagesInIntervalIteratorProvider.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/resultset/PagesInIntervalIteratorProvider.java @@ -24,6 +24,7 @@ import com.exactpro.cradle.cassandra.retries.SelectQueryExecutor; import com.exactpro.cradle.counters.Interval; import com.exactpro.cradle.utils.CradleStorageException; +import com.exactpro.cradle.utils.TimeUtils; import java.time.Instant; import java.util.LinkedList; @@ -68,7 +69,7 @@ private Queue getPagesInInterval(BookId bookId, BookCache bookCache, Int Instant start = interval.getStart(); Instant end = interval.getEnd(); - return bookCache.loadPageInfo(bookId, false) + return bookCache.loadPageInfo(bookId, TimeUtils.toLocalTimestamp(interval.getStart()), false) .stream() .filter(page -> checkInterval(page, start, end)) .map(page -> page.getId().getName()) diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java index 1c810ebcd..561b4806f 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/GroupedMessageEntityUtils.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -22,6 +22,7 @@ import com.exactpro.cradle.messages.StoredGroupedMessageBatch; import com.exactpro.cradle.messages.StoredMessage; import com.exactpro.cradle.serialization.SerializedEntityData; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.cradle.utils.CompressException; import com.exactpro.cradle.utils.CompressionType; import com.exactpro.cradle.utils.MessageUtils; @@ -39,12 +40,12 @@ public class GroupedMessageEntityUtils { private static final Logger logger = LoggerFactory.getLogger(GroupedMessageEntityUtils.class); - public static SerializedEntity toSerializedEntity( + public static SerializedEntity toSerializedEntity( GroupedMessageBatchToStore batch, PageId pageId, CompressionType compressionType, int maxUncompressedSize - ) throws IOException, CompressException { + ) throws CompressException { GroupedMessageBatchEntity.GroupedMessageBatchEntityBuilder builder = GroupedMessageBatchEntity.builder(); @@ -52,7 +53,7 @@ public static SerializedEntity toSerializedEntity( logger.debug("Creating entity from grouped message batch '{}'", group); - SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); + SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); byte[] batchContent = serializedEntityData.getSerializedData(); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java index d3fc7f9d8..23c04b729 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/utils/MessageBatchEntityUtils.java @@ -25,6 +25,7 @@ import com.exactpro.cradle.messages.StoredMessageBatch; import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.serialization.SerializedEntityData; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.cradle.utils.CompressException; import com.exactpro.cradle.utils.CompressionType; import com.exactpro.cradle.utils.MessageUtils; @@ -41,16 +42,16 @@ public class MessageBatchEntityUtils { private static final Logger logger = LoggerFactory.getLogger(MessageBatchEntityUtils.class); - public static SerializedEntity toSerializedEntity( + public static SerializedEntity toSerializedEntity( MessageBatchToStore batch, PageId pageId, CompressionType compressionType, int maxUncompressedSize - ) throws IOException, CompressException { + ) throws CompressException { logger.debug("Creating entity from message batch '{}'", batch.getId()); MessageBatchEntity.MessageBatchEntityBuilder builder = MessageBatchEntity.builder(); - SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); + SerializedEntityData serializedEntityData = MessageUtils.serializeMessages(batch); byte[] batchContent = serializedEntityData.getSerializedData(); builder.setUncompressedContentSize(batchContent.length); diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java index 9cf8ac875..e234aaf12 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/MessagesWorker.java @@ -53,7 +53,7 @@ import com.exactpro.cradle.messages.StoredMessageBatch; import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.resultset.CradleResultSet; -import com.exactpro.cradle.serialization.SerializedEntityMetadata; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.cradle.utils.CompressException; import com.exactpro.cradle.utils.CradleStorageException; import com.exactpro.cradle.utils.TimeUtils; @@ -66,13 +66,18 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.EnumMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.function.Function; +import java.util.stream.Collectors; import java.util.zip.DataFormatException; import static com.exactpro.cradle.CradleStorage.EMPTY_MESSAGE_INDEX; @@ -180,11 +185,14 @@ private static void updateMessageReadMetrics(BookId bookId, StoredGroupedMessage private static void updateMessageWriteMetrics(MessageBatchEntity entity, BookId bookId) { StreamLabel key = new StreamLabel(bookId.getName(), entity.getSessionAlias(), entity.getDirection()); - MESSAGE_STORE_METRIC.inc(key, entity.getMessageCount()); - MESSAGE_BATCH_STORE_METRIC.inc(key); - MESSAGE_STORE_UNCOMPRESSED_BYTES.inc(key, entity.getUncompressedContentSize()); + updateMessageWriteMetrics(key, entity.getMessageCount(), entity.getUncompressedContentSize()); MESSAGE_STORE_COMPRESSED_BYTES.inc(key, entity.getContentSize()); + } + private static void updateMessageWriteMetrics(StreamLabel key, int count, int uncompressedContentSize) { + MESSAGE_STORE_METRIC.inc(key, count); + MESSAGE_BATCH_STORE_METRIC.inc(key); + MESSAGE_STORE_UNCOMPRESSED_BYTES.inc(key, uncompressedContentSize); } private static void updateMessageWriteMetrics(GroupedMessageBatchEntity entity, BookId bookId) { @@ -344,33 +352,41 @@ private CompletableFuture storeGroup(GroupedMessageBatchEntity grou }, composingService); } - public CompletableFuture storePageSession(MessageBatchToStore batch, PageId pageId) { - StoredMessageId batchId = batch.getId(); + public CompletableFuture storePageSession(PageId pageId, String sessionAlias, Direction direction) { CassandraOperators operators = getOperators(); CachedPageSession cachedPageSession = new CachedPageSession(pageId.toString(), - batchId.getSessionAlias(), batchId.getDirection().getLabel()); + sessionAlias, direction.getLabel()); if (!operators.getPageSessionsCache().store(cachedPageSession)) { - logger.debug("Skipped writing page/session of message batch '{}'", batchId); + logger.debug("Skipped writing page/session of page '{}', session alias '{}', direction '{}'", pageId, sessionAlias, direction); return CompletableFuture.completedFuture(null); } - logger.debug("Writing page/session of batch '{}'", batchId); + logger.debug("Writing page/session of page '{}', session alias '{}', direction '{}'", pageId, sessionAlias, direction); - return operators.getPageSessionsOperator().write(new PageSessionEntity(batchId, pageId), writeAttrs); + return operators.getPageSessionsOperator().write( + new PageSessionEntity(pageId.getBookId().getName(), pageId.getName(), sessionAlias, direction.getLabel()), + writeAttrs + ); } - public CompletableFuture storeSession(MessageBatchToStore batch) { - StoredMessageId batchId = batch.getId(); - BookId bookId = batchId.getBookId(); + public CompletableFuture storePageSession(MessageBatchToStore batch, PageId pageId) { + return storePageSession(pageId, batch.getSessionAlias(), batch.getDirection()); + } + + public CompletableFuture storeSession(BookId bookId, String sessionAlias) { CassandraOperators operators = getOperators(); - CachedSession cachedSession = new CachedSession(bookId.toString(), batch.getSessionAlias()); + CachedSession cachedSession = new CachedSession(bookId.toString(), sessionAlias); if (!operators.getSessionsCache().store(cachedSession)) { - logger.debug("Skipped writing book/session of message batch '{}'", batchId); + logger.debug("Skipped writing book/session of book '{}', session alias '{}'", bookId, sessionAlias); return CompletableFuture.completedFuture(null); } - logger.debug("Writing book/session of batch '{}'", batchId); + logger.debug("Writing book/session of book '{}', session alias '{}'", bookId, sessionAlias); - return operators.getSessionsOperator().write(new SessionEntity(bookId.toString(), batch.getSessionAlias()), writeAttrs); + return operators.getSessionsOperator().write(new SessionEntity(bookId.toString(), sessionAlias), writeAttrs); + } + + public CompletableFuture storeSession(MessageBatchToStore batch) { + return storeSession(batch.getId().getBookId(), batch.getSessionAlias()); } public CompletableFuture storeMessageBatch(MessageBatchToStore batch, PageId pageId) { @@ -385,7 +401,7 @@ public CompletableFuture storeMessageBatch(MessageBatchToStore batch, Page } }, composingService).thenCompose(serializedEntity -> { MessageBatchEntity entity = serializedEntity.getEntity(); - List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); + List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); return mbOperator.write(entity, writeAttrs) .thenRunAsync(() -> { @@ -404,22 +420,31 @@ public CompletableFuture storeMessageBatch(MessageBatchToStore batch, Page }); } - public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToStore batchToStore, PageId pageId) { + public CompletableFuture storeGroupedMessageBatch( + GroupedMessageBatchToStore batchToStore, + PageId pageId, + boolean storeSessionMetadata + ) { BookId bookId = pageId.getBookId(); GroupedMessageBatchOperator gmbOperator = getOperators().getGroupedMessageBatchOperator(); return CompletableFuture.supplyAsync(() -> { try { - return GroupedMessageEntityUtils.toSerializedEntity(batchToStore, pageId, settings.getCompressionType(), settings.getMaxUncompressedMessageBatchSize()); + return GroupedMessageEntityUtils.toSerializedEntity( + batchToStore, + pageId, + settings.getCompressionType(), + settings.getMaxUncompressedMessageBatchSize() + ); } catch (Exception e) { throw new CompletionException(e); } }, composingService).thenCompose(serializedEntity -> { GroupedMessageBatchEntity entity = serializedEntity.getEntity(); - List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); + List meta = serializedEntity.getSerializedEntityData().getSerializedEntityMetadata(); - return gmbOperator.write(entity, writeAttrs) - .thenComposeAsync((value) -> CompletableFuture.allOf( + CompletableFuture future = gmbOperator.write(entity, writeAttrs) + .thenRunAsync(() -> CompletableFuture.allOf( storePageGroup(entity), storeGroup(entity), CompletableFuture.runAsync(() -> { @@ -436,9 +461,77 @@ public CompletableFuture storeGroupedMessageBatch(GroupedMessageBatchToSto updateMessageWriteMetrics(entity, bookId); }, composingService) ), composingService); + if (storeSessionMetadata) { + future = updateSessionStatistics(future, bookId, pageId, batchToStore, meta); + } + return future; }); } + private CompletableFuture updateSessionStatistics(CompletableFuture future, BookId bookId, PageId pageId, GroupedMessageBatchToStore batchToStore, List meta) { + return future.thenApplyAsync((unused) -> getDirectionToSessionAliases(batchToStore), composingService) + .thenComposeAsync((sessions) -> storeSessions(pageId, sessions), composingService) + .thenComposeAsync((sessions) -> storePageSessions(bookId, sessions), composingService) + .thenApplyAsync((sessions) -> { + for (Map.Entry> entry : sessions.entrySet()) { + Direction direction = entry.getKey(); + Set sessionAliases = entry.getValue(); + for (String sessionAlias : sessionAliases) { + List streamMetadatas = meta.stream() + .filter((metadata) -> + metadata.getDirection() == direction && Objects.equals(metadata.getSessionAlias(), sessionAlias) + ).collect(Collectors.toList()); + messageStatisticsCollector.updateMessageBatchStatistics(bookId, + pageId.getName(), + sessionAlias, + direction.getLabel(), + streamMetadatas); + sessionStatisticsCollector.updateSessionStatistics(bookId, + pageId.getName(), + SessionRecordType.SESSION, + sessionAlias, + streamMetadatas); + updateMessageWriteMetrics( + new StreamLabel(bookId.getName(), sessionAlias, direction.getLabel()), + streamMetadatas.size(), + streamMetadatas.stream() + .mapToInt(SerializedMessageMetadata::getSerializedEntitySize).sum() + ); + } + } + return null; + }, composingService); + } + + private static Map> getDirectionToSessionAliases(GroupedMessageBatchToStore batch) { + Map> sessions = new EnumMap<>(Direction.class); + for (Direction direction : Direction.values()) { + sessions.put(direction, new HashSet<>()); + } + for (StoredMessage message : batch.getMessages()) { + sessions.get(message.getDirection()).add(message.getSessionAlias()); + } + return sessions; + } + + private CompletableFuture>> storeSessions(PageId pageId, Map> sessions) { + return CompletableFuture.allOf( + sessions.entrySet().stream() + .flatMap((entry) -> entry.getValue().stream() + .map((sessionAlias) -> storePageSession(pageId, sessionAlias, entry.getKey())) + ).toArray(CompletableFuture[]::new) + ).thenApply((unused) -> sessions); + } + + private CompletableFuture>> storePageSessions(BookId bookId, Map> sessions) { + return CompletableFuture.allOf( + sessions.values().stream() + .flatMap(Set::stream) + .distinct() + .map((sessionAlias) -> storeSession(bookId, sessionAlias)) + .toArray(CompletableFuture[]::new) + ).thenApply((unused) -> sessions); + } public long getBoundarySequence(String sessionAlias, Direction direction, BookInfo book, boolean first) throws CradleStorageException { diff --git a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java index ee8a2c517..6141eb096 100644 --- a/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java +++ b/cradle-cassandra/src/main/java/com/exactpro/cradle/cassandra/workers/StatisticsWorker.java @@ -22,12 +22,25 @@ import com.exactpro.cradle.EntityType; import com.exactpro.cradle.FrameType; import com.exactpro.cradle.SessionRecordType; -import com.exactpro.cradle.cassandra.counters.*; +import com.exactpro.cradle.cassandra.counters.BookStatisticsRecordsCaches; +import com.exactpro.cradle.cassandra.counters.EntityStatisticsCollector; +import com.exactpro.cradle.cassandra.counters.MessageStatisticsCollector; +import com.exactpro.cradle.cassandra.counters.SessionList; +import com.exactpro.cradle.cassandra.counters.SessionStatisticsCollector; +import com.exactpro.cradle.cassandra.counters.TimeFrameRecord; +import com.exactpro.cradle.cassandra.counters.TimeFrameRecordCache; +import com.exactpro.cradle.cassandra.counters.TimeFrameRecordSamples; import com.exactpro.cradle.cassandra.dao.CassandraOperators; -import com.exactpro.cradle.cassandra.dao.statistics.*; +import com.exactpro.cradle.cassandra.dao.statistics.EntityStatisticsEntity; +import com.exactpro.cradle.cassandra.dao.statistics.EntityStatisticsOperator; +import com.exactpro.cradle.cassandra.dao.statistics.MessageStatisticsEntity; +import com.exactpro.cradle.cassandra.dao.statistics.MessageStatisticsOperator; +import com.exactpro.cradle.cassandra.dao.statistics.SessionStatisticsEntity; +import com.exactpro.cradle.cassandra.dao.statistics.SessionStatisticsOperator; import com.exactpro.cradle.cassandra.utils.LimitedCache; import com.exactpro.cradle.counters.Counter; import com.exactpro.cradle.serialization.SerializedEntityMetadata; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.th2.taskutils.FutureTracker; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,7 +49,12 @@ import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.concurrent.*; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Function; @@ -89,7 +107,7 @@ public void stop() { logger.info("Waiting statistics cache depletion"); while (bookCounterCachesNotEmpty()) try { - Thread.sleep(100); + Thread.sleep(100); // FIXME: find another way to wait the empty cache state } catch (InterruptedException e) { logger.error("Interrupted while waiting statistics cache depletion"); } @@ -100,7 +118,7 @@ public void stop() { executorService.shutdown(); try { logger.debug("Waiting StatisticsWorker jobs to complete"); - executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); + executorService.awaitTermination(Long.MAX_VALUE, TimeUnit.SECONDS); // FIXME: limit await timeout logger.debug("StatisticsWorker shutdown complete"); } catch (InterruptedException e) { logger.error("Interrupted while waiting jobs to complete"); @@ -120,7 +138,7 @@ private boolean bookCounterCachesNotEmpty() { } - private void updateCounters(TimeFrameRecordCache counters, Collection batchMetadata) { + private void updateCounters(TimeFrameRecordCache counters, Collection batchMetadata) { batchMetadata.forEach(meta -> { Counter counter = new Counter(1, meta.getSerializedEntitySize()); for (FrameType t : FrameType.values()) { @@ -131,7 +149,7 @@ private void updateCounters(TimeFrameRecordCache counters, Collection batchMetadata) { + public void updateEntityBatchStatistics(BookId bookId, BookStatisticsRecordsCaches.EntityKey entityKey, Collection batchMetadata) { if (!isEnabled) return; @@ -144,7 +162,7 @@ public void updateEntityBatchStatistics(BookId bookId, BookStatisticsRecordsCach @Override - public void updateMessageBatchStatistics(BookId bookId, String page, String sessionAlias, String direction, Collection batchMetadata) { + public void updateMessageBatchStatistics(BookId bookId, String page, String sessionAlias, String direction, Collection batchMetadata) { if (!isEnabled) return; @@ -159,7 +177,7 @@ public void updateMessageBatchStatistics(BookId bookId, String page, String sess } @Override - public void updateSessionStatistics(BookId bookId, String page, SessionRecordType recordType, String session, Collection batchMetadata) { + public void updateSessionStatistics(BookId bookId, String page, SessionRecordType recordType, String session, Collection batchMetadata) { if (!isEnabled) return; diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchEntityTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchEntityTest.java index 27234ca46..00c17b9dc 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchEntityTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/GroupedMessageBatchEntityTest.java @@ -21,7 +21,10 @@ import com.exactpro.cradle.PageId; import com.exactpro.cradle.cassandra.dao.SerializedEntity; import com.exactpro.cradle.cassandra.utils.GroupedMessageEntityUtils; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.MessageToStore; +import com.exactpro.cradle.messages.StoredGroupedMessageBatch; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.cradle.utils.CompressException; import com.exactpro.cradle.utils.CompressionType; import com.exactpro.cradle.utils.CradleStorageException; @@ -53,7 +56,7 @@ public void validationTest() throws IOException, DataFormatException, CradleStor .build(); batch.addMessage(message); - SerializedEntity serializedEntity = GroupedMessageEntityUtils.toSerializedEntity(batch, pageId, CompressionType.ZLIB, 10_000); + SerializedEntity serializedEntity = GroupedMessageEntityUtils.toSerializedEntity(batch, pageId, CompressionType.ZLIB, 10_000); StoredGroupedMessageBatch storedBatch = new StoredGroupedMessageBatch(group, batch.getMessages(), pageId, null); StoredGroupedMessageBatch batchFromEntity = GroupedMessageEntityUtils.toStoredGroupedMessageBatch(serializedEntity.getEntity(), pageId); diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/MessageBatchEntityTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/MessageBatchEntityTest.java index db3dcd129..f49c07701 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/MessageBatchEntityTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/messages/MessageBatchEntityTest.java @@ -25,6 +25,7 @@ import com.exactpro.cradle.messages.MessageBatchToStore; import com.exactpro.cradle.messages.MessageToStore; import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import com.exactpro.cradle.utils.CompressException; import com.exactpro.cradle.utils.CompressionType; import com.exactpro.cradle.utils.CradleStorageException; @@ -53,7 +54,7 @@ public void messageEntity() throws IOException, DataFormatException, CradleStora .build(), 200, new CoreStorageSettings().calculateStoreActionRejectionThreshold()); StoredMessageBatch storedBatch = new StoredMessageBatch(batch.getMessages(), pageId, null); - SerializedEntity serializedEntity = MessageBatchEntityUtils.toSerializedEntity(batch, pageId, CompressionType.ZLIB, 2000); + SerializedEntity serializedEntity = MessageBatchEntityUtils.toSerializedEntity(batch, pageId, CompressionType.ZLIB, 2000); StoredMessageBatch batchFromEntity = MessageBatchEntityUtils.toStoredMessageBatch(serializedEntity.getEntity(), pageId); RecursiveComparisonConfiguration config = new RecursiveComparisonConfiguration(); diff --git a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityTest.java b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityTest.java index 799a98854..af7b1c52f 100644 --- a/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityTest.java +++ b/cradle-cassandra/src/test/java/com/exactpro/cradle/cassandra/dao/testevents/TestEventEntityTest.java @@ -22,6 +22,7 @@ import com.exactpro.cradle.PageId; import com.exactpro.cradle.cassandra.dao.SerializedEntity; import com.exactpro.cradle.messages.StoredMessageId; +import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.testevents.StoredTestEvent; import com.exactpro.cradle.testevents.StoredTestEventId; import com.exactpro.cradle.testevents.TestEventBatchToStore; @@ -93,7 +94,7 @@ private Set createMessageIds(int size) { @Test(dataProvider = "events") public void eventEntity(TestEventToStore event) throws CradleStorageException, IOException, DataFormatException, CradleIdException, CompressException { - SerializedEntity serializedEntity = TestEventEntityUtils.toSerializedEntity(event, page, CompressionType.ZLIB, 2000); + SerializedEntity serializedEntity = TestEventEntityUtils.toSerializedEntity(event, page, CompressionType.ZLIB, 2000); TestEventEntity entity = serializedEntity.getEntity(); StoredTestEvent newEvent = TestEventEntityUtils.toStoredTestEvent(entity, page); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/BookCache.java b/cradle-core/src/main/java/com/exactpro/cradle/BookCache.java index 03776ce4d..7edd8849c 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/BookCache.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/BookCache.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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,6 +17,7 @@ import com.exactpro.cradle.utils.CradleStorageException; +import java.time.LocalDateTime; import java.util.Collection; /* @@ -27,11 +28,17 @@ */ public interface BookCache { - BookInfo getBook (BookId bookId) throws CradleStorageException; - boolean checkBook (BookId bookId); + BookInfo getBook(BookId bookId) throws CradleStorageException; + + boolean checkBook(BookId bookId); + Collection loadPageInfo(BookId bookId, boolean loadRemoved) throws CradleStorageException; - BookInfo loadBook (BookId bookId) throws CradleStorageException; - void updateCachedBook (BookInfo bookInfo); - Collection getCachedBooks (); + Collection loadPageInfo(BookId bookId, LocalDateTime start, boolean loadRemoved) throws CradleStorageException; + + BookInfo loadBook(BookId bookId) throws CradleStorageException; + + void updateCachedBook(BookInfo bookInfo); + + Collection getCachedBooks(); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CoreStorageSettings.java b/cradle-core/src/main/java/com/exactpro/cradle/CoreStorageSettings.java index 93d99c1af..f0b9b30a9 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CoreStorageSettings.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CoreStorageSettings.java @@ -16,10 +16,21 @@ package com.exactpro.cradle; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.MessageBatchToStore; + @SuppressWarnings("unused") public class CoreStorageSettings { public static final long DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS = 60000; - protected long bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; + public static final boolean DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS = true; + private long bookRefreshIntervalMillis = DEFAULT_BOOK_REFRESH_INTERVAL_MILLIS; + + /** + * if `true`, the {@link CradleStorage#storeGroupedMessageBatch} method converts {@link GroupedMessageBatchToStore} to {@link MessageBatchToStore} and stores them. + * if `false`, the {@link CradleStorage#storeGroupedMessageBatch} method stores only {@link GroupedMessageBatchToStore}, + * also the {@link CradleStorage#storeMessageBatch} and {@link CradleStorage#storeMessageBatchAsync} methods throws {@link IllegalStateException} + */ + private boolean storeIndividualMessageSessions = DEFAULT_STORE_INDIVIDUAL_MESSAGE_SESSIONS; public long getBookRefreshIntervalMillis() { return bookRefreshIntervalMillis; @@ -36,4 +47,12 @@ public long calculatePageActionRejectionThreshold() { public long calculateStoreActionRejectionThreshold() { return this.bookRefreshIntervalMillis; } + + public boolean isStoreIndividualMessageSessions() { + return storeIndividualMessageSessions; + } + + public void setStoreIndividualMessageSessions(boolean storeIndividualMessageSessions) { + this.storeIndividualMessageSessions = storeIndividualMessageSessions; + } } \ No newline at end of file diff --git a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java index f4a3f8e4b..5bfffb4a8 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/CradleStorage.java @@ -90,6 +90,7 @@ public abstract class CradleStorage { private final long pageActionRejectionThreshold; private final long storeActionRejectionThreshold; + private final boolean storeIndividualMessageSessions; public CradleStorage( ExecutorService composingService, @@ -109,6 +110,7 @@ public CradleStorage( this.pageActionRejectionThreshold = settings.calculatePageActionRejectionThreshold(); this.storeActionRejectionThreshold = settings.calculateStoreActionRejectionThreshold(); + this.storeIndividualMessageSessions = settings.isStoreIndividualMessageSessions(); entitiesFactory = new CradleEntitiesFactory(maxMessageBatchSize, maxTestEventBatchSize, storeActionRejectionThreshold); } @@ -576,9 +578,13 @@ public CradleEntitiesFactory getEntitiesFactory() { * @param batch data to write * @throws IOException if data writing failed * @throws CradleStorageException if given parameters are invalid + * @throws IllegalStateException if store individual message sessions is false */ @Deprecated public final void storeMessageBatch(MessageBatchToStore batch) throws IOException, CradleStorageException { + if (!storeIndividualMessageSessions) { + throw new IllegalStateException("Message batch can't be stored when store individual message sessions is false"); + } StoredMessageId id = batch.getId(); logger.debug("Storing message batch {}", id); PageInfo page = findPage(id.getBookId(), id.getTimestamp()); @@ -609,10 +615,14 @@ public final void storeGroupedMessageBatch(GroupedMessageBatchToStore batch) * @return future to get know if storing was successful * @throws CradleStorageException if given parameters are invalid * @throws IOException if data writing failed + * @throws IllegalStateException if store individual message sessions is false */ @Deprecated public final CompletableFuture storeMessageBatchAsync(MessageBatchToStore batch) throws CradleStorageException, IOException { + if (!storeIndividualMessageSessions) { + throw new IllegalStateException("Message batch can't be stored when store individual message sessions is false"); + } StoredMessageId id = batch.getId(); logger.debug("Storing message batch {} asynchronously", id); PageInfo page = findPage(id.getBookId(), id.getTimestamp()); @@ -1157,9 +1167,8 @@ public final CradleResultSet getTestEvents(TestEventFilter filt * @param filter defines conditions to filter test events by * @return future to obtain result set to enumerate test events * @throws CradleStorageException if filter is invalid - * @throws IOException if data retrieval failed */ - public final CompletableFuture> getTestEventsAsync(TestEventFilter filter) throws CradleStorageException, IOException { + public final CompletableFuture> getTestEventsAsync(TestEventFilter filter) throws CradleStorageException { logger.debug("Asynchronously getting test events filtered by {}", filter); if (!checkFilter(filter)) return CompletableFuture.completedFuture(new EmptyResultSet<>()); @@ -1423,9 +1432,8 @@ public CradleResultSet getSessionGroups(BookId bookId, Interval interval * @param comment updated comment value for page * @return returns PageInfo of updated page * @throws CradleStorageException Page was edited but cache wasn't refreshed, try to refresh pages - * @throws IOException if there is a problem with input/output */ - public PageInfo updatePageComment(BookId bookId, String pageName, String comment) throws CradleStorageException, IOException { + public PageInfo updatePageComment(BookId bookId, String pageName, String comment) throws CradleStorageException { getBookCache().getBook(bookId); PageInfo updatedPageInfo = doUpdatePageComment(bookId, pageName, comment); @@ -1447,9 +1455,8 @@ public PageInfo updatePageComment(BookId bookId, String pageName, String comment * @param newPageName name after update * @return returns PageInfo of updated page * @throws CradleStorageException Page was edited but cache wasn't refreshed, try to refresh pages - * @throws IOException if there is a problem with input/output */ - public PageInfo updatePageName(BookId bookId, String pageName, String newPageName) throws CradleStorageException, IOException { + public PageInfo updatePageName(BookId bookId, String pageName, String newPageName) throws CradleStorageException { getBookCache().getBook(bookId); PageInfo updatedPageInfo = doUpdatePageName(bookId, pageName, newPageName); @@ -1483,7 +1490,7 @@ public Iterator getPages(BookId bookId, Interval interval) throws Crad * @param interval Time interval * @return iterator of PageInfo */ - public CompletableFuture> getPagesAsync(BookId bookId, Interval interval) throws CradleStorageException { + public CompletableFuture> getPagesAsync(BookId bookId, Interval interval) { return doGetPagesAsync(bookId, interval); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/DummyCradleStorage.java b/cradle-core/src/main/java/com/exactpro/cradle/DummyCradleStorage.java index 42a1abe64..5defcbd77 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/DummyCradleStorage.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/DummyCradleStorage.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2021 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -16,36 +16,42 @@ package com.exactpro.cradle; -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; - import com.exactpro.cradle.counters.Counter; import com.exactpro.cradle.counters.CounterSample; import com.exactpro.cradle.counters.Interval; import com.exactpro.cradle.intervals.IntervalsWorker; -import com.exactpro.cradle.messages.*; +import com.exactpro.cradle.messages.GroupedMessageBatchToStore; +import com.exactpro.cradle.messages.GroupedMessageFilter; +import com.exactpro.cradle.messages.MessageBatchToStore; +import com.exactpro.cradle.messages.MessageFilter; +import com.exactpro.cradle.messages.StoredGroupedMessageBatch; +import com.exactpro.cradle.messages.StoredMessage; +import com.exactpro.cradle.messages.StoredMessageBatch; +import com.exactpro.cradle.messages.StoredMessageId; import com.exactpro.cradle.resultset.CradleResultSet; import com.exactpro.cradle.testevents.StoredTestEvent; -import com.exactpro.cradle.testevents.TestEventFilter; import com.exactpro.cradle.testevents.StoredTestEventId; +import com.exactpro.cradle.testevents.TestEventFilter; import com.exactpro.cradle.testevents.TestEventToStore; import com.exactpro.cradle.utils.CradleStorageException; +import java.time.LocalDateTime; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; + /** * Dummy implementation of CradleStorage that does nothing and serves as a stub */ -public class DummyCradleStorage extends CradleStorage -{ - class DummyBookCache implements BookCache { +public class DummyCradleStorage extends CradleStorage { + static class DummyBookCache implements BookCache { Map books; - DummyBookCache () { + DummyBookCache() { books = new ConcurrentHashMap<>(); } @@ -63,12 +69,17 @@ public boolean checkBook(BookId bookId) { } @Override - public Collection loadPageInfo(BookId bookId, boolean loadRemoved) throws CradleStorageException { + public Collection loadPageInfo(BookId bookId, boolean loadRemoved) { return null; } @Override - public BookInfo loadBook(BookId bookId) throws CradleStorageException { + public Collection loadPageInfo(BookId bookId, LocalDateTime start, boolean loadRemoved) { + return null; + } + + @Override + public BookInfo loadBook(BookId bookId) { return null; } @@ -95,16 +106,14 @@ public DummyCradleStorage() throws CradleStorageException super(); dummyBookCache = new DummyBookCache(); } - - + + @Override - protected void doInit(boolean prepareStorage) throws CradleStorageException - { + protected void doInit(boolean prepareStorage) { } - + @Override - protected void doDispose() throws CradleStorageException - { + protected void doDispose() { } @Override @@ -121,34 +130,27 @@ protected Collection doListBooks() { protected void doAddBook(BookToAdd newBook, BookId bookId) { } - + @Override - protected void doAddPages(BookId bookId, List pages, PageInfo lastPage) - throws CradleStorageException, IOException - { + protected void doAddPages(BookId bookId, List pages, PageInfo lastPage) { } - + @Override - protected Collection doLoadPages(BookId bookId) throws CradleStorageException, IOException - { + protected Collection doLoadPages(BookId bookId) { return null; } - + @Override - protected void doRemovePage(PageInfo page) throws CradleStorageException, IOException - { + protected void doRemovePage(PageInfo page) { } - + @Override - protected void doStoreMessageBatch(MessageBatchToStore batch, PageInfo page) throws IOException - { + protected void doStoreMessageBatch(MessageBatchToStore batch, PageInfo page) { } @Override - protected void doStoreGroupedMessageBatch(GroupedMessageBatchToStore batch, PageInfo page) - throws IOException - { - + protected void doStoreGroupedMessageBatch(GroupedMessageBatchToStore batch, PageInfo page) { + } @Override @@ -159,15 +161,12 @@ protected CompletableFuture doStoreMessageBatchAsync(MessageBatchToStore b } @Override - protected CompletableFuture doStoreGroupedMessageBatchAsync(GroupedMessageBatchToStore batch, PageInfo page) - throws IOException, CradleStorageException - { + protected CompletableFuture doStoreGroupedMessageBatchAsync(GroupedMessageBatchToStore batch, PageInfo page) { return null; } @Override - protected void doStoreTestEvent(TestEventToStore event, PageInfo page) throws IOException - { + protected void doStoreTestEvent(TestEventToStore event, PageInfo page) { } @Override @@ -175,10 +174,9 @@ protected CompletableFuture doStoreTestEventAsync(TestEventToStore event, { return CompletableFuture.completedFuture(null); } - + @Override - protected void doUpdateParentTestEvents(TestEventToStore event) throws IOException - { + protected void doUpdateParentTestEvents(TestEventToStore event) { } @Override @@ -186,10 +184,9 @@ protected CompletableFuture doUpdateParentTestEventsAsync(TestEventToStore { return CompletableFuture.completedFuture(null); } - + @Override - protected void doUpdateEventStatus(StoredTestEvent event, boolean success) throws IOException - { + protected void doUpdateEventStatus(StoredTestEvent event, boolean success) { } @Override @@ -197,11 +194,10 @@ protected CompletableFuture doUpdateEventStatusAsync(StoredTestEvent event { return CompletableFuture.completedFuture(null); } - - + + @Override - protected StoredMessage doGetMessage(StoredMessageId id, PageId pageId) throws IOException - { + protected StoredMessage doGetMessage(StoredMessageId id, PageId pageId) { return null; } @@ -210,10 +206,9 @@ protected CompletableFuture doGetMessageAsync(StoredMessageId id, { return CompletableFuture.completedFuture(null); } - + @Override - protected StoredMessageBatch doGetMessageBatch(StoredMessageId id, PageId pageId) throws IOException - { + protected StoredMessageBatch doGetMessageBatch(StoredMessageId id, PageId pageId) { return null; } @@ -222,10 +217,9 @@ protected CompletableFuture doGetMessageBatchAsync(StoredMes { return CompletableFuture.completedFuture(null); } - + @Override - protected CradleResultSet doGetMessages(MessageFilter filter, BookInfo book) throws IOException - { + protected CradleResultSet doGetMessages(MessageFilter filter, BookInfo book) { return null; } @@ -235,18 +229,15 @@ protected CompletableFuture> doGetMessagesAsync(M { return CompletableFuture.completedFuture(null); } - + @Override - protected CradleResultSet doGetMessageBatches(MessageFilter filter, BookInfo book) throws IOException - { + protected CradleResultSet doGetMessageBatches(MessageFilter filter, BookInfo book) { return null; } @Override protected CradleResultSet doGetGroupedMessageBatches(GroupedMessageFilter filter, - BookInfo book) - throws IOException, CradleStorageException - { + BookInfo book) { return null; } @@ -259,40 +250,34 @@ protected CompletableFuture> doGetMessageBat @Override protected CompletableFuture> doGetGroupedMessageBatchesAsync( - GroupedMessageFilter filter, BookInfo book) throws CradleStorageException - { + GroupedMessageFilter filter, BookInfo book) { return null; } @Override - protected long doGetLastSequence(String sessionAlias, Direction direction, BookId bookId) throws IOException - { + protected long doGetLastSequence(String sessionAlias, Direction direction, BookId bookId) { return 0; } @Override - protected long doGetFirstSequence(String sessionAlias, Direction direction, BookId bookId) - throws IOException, CradleStorageException - { + protected long doGetFirstSequence(String sessionAlias, Direction direction, BookId bookId) { return 0; } @Override - protected Collection doGetSessionAliases(BookId bookId) throws IOException - { + protected Collection doGetSessionAliases(BookId bookId) { return null; } @Override - protected Collection doGetGroups(BookId bookId) throws IOException, CradleStorageException { + protected Collection doGetGroups(BookId bookId) { return null; } @Override - protected StoredTestEvent doGetTestEvent(StoredTestEventId id, PageId pageId) throws IOException - { + protected StoredTestEvent doGetTestEvent(StoredTestEventId id, PageId pageId) { return null; } @@ -301,105 +286,100 @@ protected CompletableFuture doGetTestEventAsync(StoredTestEvent { return CompletableFuture.completedFuture(null); } - + @Override - protected CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) - throws CradleStorageException, IOException - { + protected CradleResultSet doGetTestEvents(TestEventFilter filter, BookInfo book) { return null; } - + @Override - protected CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) - throws CradleStorageException - { + protected CompletableFuture> doGetTestEventsAsync(TestEventFilter filter, BookInfo book) { return CompletableFuture.completedFuture(null); } - - + + @Override - protected Collection doGetScopes(BookId bookId) throws IOException, CradleStorageException - { + protected Collection doGetScopes(BookId bookId) { return null; } @Override - protected CradleResultSet doGetScopes(BookId bookId, Interval interval) throws CradleStorageException { + protected CradleResultSet doGetScopes(BookId bookId, Interval interval) { return null; } @Override - protected CompletableFuture> doGetScopesAsync(BookId bookId, Interval interval) throws CradleStorageException { + protected CompletableFuture> doGetScopesAsync(BookId bookId, Interval interval) { return null; } @Override - protected CompletableFuture> doGetMessageCountersAsync(BookId bookId, String sessionAlias, Direction direction, FrameType frameType, Interval interval) throws CradleStorageException { + protected CompletableFuture> doGetMessageCountersAsync(BookId bookId, String sessionAlias, Direction direction, FrameType frameType, Interval interval) { return null; } @Override - protected CradleResultSet doGetMessageCounters(BookId bookId, String sessionAlias, Direction direction, FrameType frameType, Interval interval) throws CradleStorageException, IOException { + protected CradleResultSet doGetMessageCounters(BookId bookId, String sessionAlias, Direction direction, FrameType frameType, Interval interval) { return null; } @Override - protected CompletableFuture> doGetCountersAsync(BookId bookId, EntityType entityType, FrameType frameType, Interval interval) throws CradleStorageException { + protected CompletableFuture> doGetCountersAsync(BookId bookId, EntityType entityType, FrameType frameType, Interval interval) { return null; } @Override - protected CradleResultSet doGetCounters(BookId bookId, EntityType entityType, FrameType frameType, Interval interval) throws CradleStorageException, IOException { + protected CradleResultSet doGetCounters(BookId bookId, EntityType entityType, FrameType frameType, Interval interval) { return null; } @Override - protected CompletableFuture doGetMessageCountAsync(BookId bookId, String sessionAlias, Direction direction, Interval interval) throws CradleStorageException { + protected CompletableFuture doGetMessageCountAsync(BookId bookId, String sessionAlias, Direction direction, Interval interval) { return null; } @Override - protected Counter doGetMessageCount(BookId bookId, String sessionAlias, Direction direction, Interval interval) throws CradleStorageException, IOException { + protected Counter doGetMessageCount(BookId bookId, String sessionAlias, Direction direction, Interval interval) { return null; } @Override - protected CompletableFuture doGetCountAsync(BookId bookId, EntityType entityType, Interval interval) throws CradleStorageException { + protected CompletableFuture doGetCountAsync(BookId bookId, EntityType entityType, Interval interval) { return null; } @Override - protected Counter doGetCount(BookId bookId, EntityType entityType, Interval interval) throws CradleStorageException, IOException { + protected Counter doGetCount(BookId bookId, EntityType entityType, Interval interval) { return null; } @Override - protected CompletableFuture> doGetSessionAliasesAsync(BookId bookId, Interval interval) throws CradleStorageException { + protected CompletableFuture> doGetSessionAliasesAsync(BookId bookId, Interval interval) { return null; } @Override - protected CradleResultSet doGetSessionAliases(BookId bookId, Interval interval) throws CradleStorageException { + protected CradleResultSet doGetSessionAliases(BookId bookId, Interval interval) { return null; } @Override - protected CompletableFuture> doGetSessionGroupsAsync(BookId bookId, Interval interval) throws CradleStorageException { + protected CompletableFuture> doGetSessionGroupsAsync(BookId bookId, Interval interval) { return null; } @Override - protected CradleResultSet doGetSessionGroups(BookId bookId, Interval interval) throws CradleStorageException { + protected CradleResultSet doGetSessionGroups(BookId bookId, Interval interval) { return null; } @Override - protected PageInfo doUpdatePageComment(BookId bookId, String pageName, String comment) throws CradleStorageException { + protected PageInfo doUpdatePageComment(BookId bookId, String pageName, String comment) { return null; } @Override - protected PageInfo doUpdatePageName(BookId bookId, String pageName, String newPageName) throws CradleStorageException { + protected PageInfo doUpdatePageName(BookId bookId, String pageName, String newPageName) { return null; } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/PageInfo.java b/cradle-core/src/main/java/com/exactpro/cradle/PageInfo.java index 1ce36303b..240f82469 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/PageInfo.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/PageInfo.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -102,4 +102,16 @@ && getStarted().equals(pageInfo.getStarted()) && Objects.equals(getUpdated(), pageInfo.getUpdated()) && Objects.equals(getRemoved(), pageInfo.getRemoved()); } + + @Override + public String toString() { + return "PageInfo{" + + "id=" + id + + ", started=" + started + + ", ended=" + ended + + ", comment='" + comment + '\'' + + ", updated=" + updated + + ", removed=" + removed + + '}'; + } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/counters/Counter.java b/cradle-core/src/main/java/com/exactpro/cradle/counters/Counter.java index a9a9d2477..a18f28744 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/counters/Counter.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/counters/Counter.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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. @@ -55,4 +55,12 @@ public int hashCode() { result = 31 * result + (int) (getEntitySize() ^ (getEntitySize() >>> 32)); return result; } + + @Override + public String toString() { + return "Counter{" + + "entityCount=" + entityCount + + ", entitySize=" + entitySize + + '}'; + } } \ No newline at end of file diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/EventBatchSerializer.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/EventBatchSerializer.java index 6723cff36..209aedce9 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/EventBatchSerializer.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/EventBatchSerializer.java @@ -1,5 +1,5 @@ /* - * Copyright 2021-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2021-2023 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. @@ -29,13 +29,18 @@ import static com.exactpro.cradle.serialization.EventsSizeCalculator.calculateBatchEventSize; import static com.exactpro.cradle.serialization.EventsSizeCalculator.calculateEventRecordSize; -import static com.exactpro.cradle.serialization.Serialization.EventBatchConst.*; -import static com.exactpro.cradle.serialization.SerializationUtils.*; +import static com.exactpro.cradle.serialization.Serialization.EventBatchConst.EVENT_BATCH_ENT_MAGIC; +import static com.exactpro.cradle.serialization.Serialization.EventBatchConst.EVENT_BATCH_MAGIC; +import static com.exactpro.cradle.serialization.Serialization.EventBatchConst.EVENT_BATCH_PROTOCOL_VER; +import static com.exactpro.cradle.serialization.SerializationUtils.printBody; +import static com.exactpro.cradle.serialization.SerializationUtils.printInstant; +import static com.exactpro.cradle.serialization.SerializationUtils.printSingleBoolean; +import static com.exactpro.cradle.serialization.SerializationUtils.printString; public class EventBatchSerializer { - public byte[] serializeEventRecord (BatchedStoredTestEvent event) throws SerializationException { + public byte[] serializeEventRecord(BatchedStoredTestEvent event) { ByteBuffer allocate = ByteBuffer.allocate(calculateEventRecordSize(event)); this.serializeEventRecord(event, allocate); return allocate.array(); @@ -51,8 +56,8 @@ private void printId(StoredTestEventId id, ByteBuffer buffer) { printInstant(start, buffer); printString(id_str, buffer); } - - public void serializeEventRecord (BatchedStoredTestEvent event, ByteBuffer buffer) throws SerializationException { + + public void serializeEventRecord(BatchedStoredTestEvent event, ByteBuffer buffer) { buffer.putShort(EVENT_BATCH_ENT_MAGIC); printId(event.getId(), buffer); @@ -65,22 +70,22 @@ public void serializeEventRecord (BatchedStoredTestEvent event, ByteBuffer buffe } - public SerializedEntityData serializeEventBatch (Collection batch) throws SerializationException { + public SerializedEntityData serializeEventBatch(Collection batch) { SerializationBatchSizes sizes = calculateBatchEventSize(batch); ByteBuffer buffer = ByteBuffer.allocate(sizes.total); List serializedEventMetadata = serializeEventBatch(batch, buffer, sizes); - return new SerializedEntityData(serializedEventMetadata, buffer.array()); + return new SerializedEntityData<>(serializedEventMetadata, buffer.array()); } - public void serializeEventBatch (Collection batch, ByteBuffer buffer) throws SerializationException { + public void serializeEventBatch(Collection batch, ByteBuffer buffer) { SerializationBatchSizes eventBatchSizes = calculateBatchEventSize(batch); serializeEventBatch(batch, buffer, eventBatchSizes); } - public List serializeEventBatch ( + public List serializeEventBatch( Collection batch, ByteBuffer buffer, SerializationBatchSizes eventBatchSizes - ) throws SerializationException { + ) { List serializedEventMetadata = new ArrayList<>(batch.size()); @@ -101,12 +106,12 @@ public List serializeEventBatch ( return Collections.unmodifiableList(serializedEventMetadata); } - public SerializedEntityData serializeEvent(TestEventSingleToStore testEvent) { + public SerializedEntityData serializeEvent(TestEventSingleToStore testEvent) { byte[] eventContent = testEvent.getContent(); SerializedEntityMetadata serializedEventMetadata = new SerializedEntityMetadata( testEvent.getStartTimestamp(), eventContent == null ? 0 : eventContent.length ); - return new SerializedEntityData(Collections.singletonList(serializedEventMetadata), eventContent); + return new SerializedEntityData<>(Collections.singletonList(serializedEventMetadata), eventContent); } } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java index 81460cae7..3031dc5a3 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/MessageSerializer.java @@ -37,52 +37,50 @@ import static com.exactpro.cradle.serialization.SerializationUtils.printString; public class MessageSerializer { - public SerializedEntityData serializeBatch(MessageBatchToStore batch) throws SerializationException { + public SerializedEntityData serializeBatch(MessageBatchToStore batch) { return serializeBatch(batch.getMessages(), batch.getBatchSize()); } - public SerializedEntityData serializeBatch(GroupedMessageBatchToStore batch) throws SerializationException { + public SerializedEntityData serializeBatch(GroupedMessageBatchToStore batch) { return serializeBatch(batch.getMessages(), batch.getBatchSize()); } - public SerializedEntityData serializeBatch(Collection batch) throws SerializationException { + public SerializedEntityData serializeBatch(Collection batch) { return serializeBatch(batch, MessagesSizeCalculator.calculateMessageBatchSize(batch)); } - private SerializedEntityData serializeBatch(Collection batch, int batchSize) throws SerializationException { + private SerializedEntityData serializeBatch(Collection batch, int batchSize) { ByteBuffer buffer = ByteBuffer.allocate(batchSize); - List serializedMessageMetadata = this.serializeBatch(batch, buffer); - return new SerializedEntityData(serializedMessageMetadata, buffer.array()); + List serializedMessageMetadata = this.serializeBatch(batch, buffer); + return new SerializedEntityData<>(serializedMessageMetadata, buffer.array()); } - public List serializeBatch( + public List serializeBatch( Collection batch, ByteBuffer buffer - ) throws SerializationException { - List serializedMessageMetadata = new ArrayList<>(batch.size()); + ) { + List serializedMessageMetadata = new ArrayList<>(batch.size()); buffer.putInt(MESSAGE_BATCH_MAGIC); buffer.put(MESSAGE_PROTOCOL_VER); buffer.putInt(batch.size()); - int i = 0; for (StoredMessage message : batch) { int messageSize = message.getSerializedSize() - MESSAGE_LENGTH_IN_BATCH; buffer.putInt(messageSize); this.serialize(message, buffer); - serializedMessageMetadata.add(new SerializedEntityMetadata(message.getTimestamp(), messageSize)); - i++; + serializedMessageMetadata.add(new SerializedMessageMetadata(message.getSessionAlias(), message.getDirection(), message.getTimestamp(), messageSize)); } return serializedMessageMetadata; } - - public byte[] serialize(StoredMessage message) throws SerializationException { + + public byte[] serialize(StoredMessage message) { ByteBuffer b = ByteBuffer.allocate(MessagesSizeCalculator.calculateMessageSize(message)); this.serialize(message, b); return b.array(); } - public void serialize(StoredMessage message, ByteBuffer buffer) throws SerializationException { + public void serialize(StoredMessage message, ByteBuffer buffer) { buffer.putShort(MESSAGE_MAGIC); StoredMessageId id = message.getId(); printString(id.getSessionAlias(), buffer); diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedEntityData.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedEntityData.java index 02878b67f..dbf6744d3 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedEntityData.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedEntityData.java @@ -1,5 +1,5 @@ /* - * Copyright 2020-2022 Exactpro (Exactpro Systems Limited) + * Copyright 2020-2023 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,16 +17,16 @@ import java.util.List; -public class SerializedEntityData { - private final List serializedEntityMetadata; +public class SerializedEntityData { + private final List serializedEntityMetadata; private final byte[] serializedData; - public SerializedEntityData(List serializedEntityMetadata, byte[] serializedData) { + public SerializedEntityData(List serializedEntityMetadata, byte[] serializedData) { this.serializedEntityMetadata = serializedEntityMetadata; this.serializedData = serializedData; } - public List getSerializedEntityMetadata() { + public List getSerializedEntityMetadata() { return serializedEntityMetadata; } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedMessageMetadata.java b/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedMessageMetadata.java new file mode 100644 index 000000000..b0818f83a --- /dev/null +++ b/cradle-core/src/main/java/com/exactpro/cradle/serialization/SerializedMessageMetadata.java @@ -0,0 +1,40 @@ +/* + * Copyright 2023 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.cradle.serialization; + +import com.exactpro.cradle.Direction; + +import java.time.Instant; + +public class SerializedMessageMetadata extends SerializedEntityMetadata { + + private final String sessionAlias; + private final Direction direction; + + public SerializedMessageMetadata(String sessionAlias, Direction direction, Instant start, int serializedEntitySize) { + super(start, serializedEntitySize); + this.sessionAlias = sessionAlias; + this.direction = direction; + } + + public String getSessionAlias() { + return sessionAlias; + } + + public Direction getDirection() { + return direction; + } +} diff --git a/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java b/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java index c1bcb6d3c..ce537fb9a 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/utils/MessageUtils.java @@ -26,6 +26,7 @@ import com.exactpro.cradle.serialization.MessageDeserializer; import com.exactpro.cradle.serialization.MessageSerializer; import com.exactpro.cradle.serialization.SerializedEntityData; +import com.exactpro.cradle.serialization.SerializedMessageMetadata; import org.apache.commons.lang3.ArrayUtils; import org.apache.commons.lang3.StringUtils; @@ -66,9 +67,8 @@ public static void validateMessage(CradleMessage message) throws CradleStorageEx * * @param batch to serialize * @return {@link SerializedEntityData} containing serialized messages. - * @throws IOException if serialization failed */ - public static SerializedEntityData serializeMessages(MessageBatchToStore batch) throws IOException { + public static SerializedEntityData serializeMessages(MessageBatchToStore batch) { return serializer.serializeBatch(batch); } @@ -77,19 +77,18 @@ public static SerializedEntityData serializeMessages(MessageBatchToStore batch) * * @param batch to serialize * @return {@link SerializedEntityData} containing serialized messages. - * @throws IOException if serialization failed */ - public static SerializedEntityData serializeMessages(GroupedMessageBatchToStore batch) throws IOException { + public static SerializedEntityData serializeMessages(GroupedMessageBatchToStore batch) { return serializer.serializeBatch(batch); } /** * Serializes messages, skipping non-meaningful or calculable fields + * * @param messages to serialize * @return {@link SerializedEntityData} containing serialized messages. - * @throws IOException if serialization failed */ - public static SerializedEntityData serializeMessages(Collection messages) throws IOException { + public static SerializedEntityData serializeMessages(Collection messages) { return serializer.serializeBatch(messages); } diff --git a/cradle-core/src/main/java/com/exactpro/cradle/utils/TestEventUtils.java b/cradle-core/src/main/java/com/exactpro/cradle/utils/TestEventUtils.java index a02ae0f66..a21efc72e 100644 --- a/cradle-core/src/main/java/com/exactpro/cradle/utils/TestEventUtils.java +++ b/cradle-core/src/main/java/com/exactpro/cradle/utils/TestEventUtils.java @@ -26,6 +26,7 @@ import com.exactpro.cradle.serialization.EventMessageIdDeserializer; import com.exactpro.cradle.serialization.EventMessageIdSerializer; import com.exactpro.cradle.serialization.SerializedEntityData; +import com.exactpro.cradle.serialization.SerializedEntityMetadata; import com.exactpro.cradle.testevents.BatchedStoredTestEvent; import com.exactpro.cradle.testevents.StoredTestEventId; import com.exactpro.cradle.testevents.TestEvent; @@ -125,9 +126,8 @@ public static void validateTestEventEndDate(TestEvent event) throws CradleStorag * * @param testEvents to serialize * @return array of bytes, containing serialized events - * @throws IOException if serialization failed */ - public static SerializedEntityData serializeTestEvents(Collection testEvents) throws IOException { + public static SerializedEntityData serializeTestEvents(Collection testEvents) { return serializer.serializeEventBatch(testEvents); } @@ -137,7 +137,7 @@ public static SerializedEntityData serializeTestEvents(Collection serializeTestEvent(TestEventSingleToStore testEvent) { return serializer.serializeEvent(testEvent); } @@ -184,9 +184,8 @@ public static byte[] getTestEventContentBytes(ByteBuffer content, boolean compre * * @param event whose content to get * @return {@link SerializedEntityData} containing test event content. - * @throws IOException if batch children serialization failed */ - public static SerializedEntityData getTestEventContent(TestEventToStore event) throws IOException { + public static SerializedEntityData getTestEventContent(TestEventToStore event) { if (event.isBatch()) { logger.trace("Serializing children of test event batch '{}'", event.getId()); return serializeTestEvents(event.asBatch().getTestEvents());