c) throws InterruptedException;
54 |
55 | /**
56 | * Retrieves and removes the first message of this buffer, waiting up to the specified wait time if necessary for a
57 | * message to become available.
58 | *
59 | * @param timeout how long to wait before giving up, in units of {@code unit}
60 | * @param unit a {@link TimeUnit} determining how to interpret the {@code timeout} parameter
61 | * @return the first message of this buffer, or {@code null} if the specified waiting time elapses before a message
62 | * is available
63 | * @throws InterruptedException if interrupted while waiting to poll an element
64 | */
65 | E poll(long timeout, TimeUnit unit) throws InterruptedException;
66 |
67 | /**
68 | * Retrieves and removes the first messages of this buffer, waiting if necessary until a message becomes available.
69 | *
70 | * @return the first message of this buffer
71 | * @throws InterruptedException if interrupted while waiting to take the first element
72 | */
73 | E take() throws InterruptedException;
74 |
75 | /**
76 | * Retrieves, but does not remove, the first message of this buffer, or returns {@code null} if this buffer is
77 | * empty.
78 | *
79 | * @return the first message in this buffer or {@code null} if the buffer is empty
80 | */
81 | E peek();
82 |
83 | /**
84 | * Returns the number of elements in this buffer.
85 | *
86 | * @return the number of elements in this buffer
87 | */
88 | int size();
89 |
90 | /**
91 | * Verify whether this buffer is empty or not.
92 | *
93 | * @return {@code true} if the buffer is empty and {@code false} if it isn't
94 | */
95 | boolean isEmpty();
96 |
97 | /**
98 | * Returns the number of additional elements that this buffer can ideally (in the absence of memory or resource
99 | * constraints) accept without blocking. This is always equal to the initial capacity of this buffer less the
100 | * current {@code size} of this buffer.
101 | *
102 | * Note that you cannot always tell if an attempt to insert an element will succeed by inspecting {@code
103 | * remainingCapacity} because it may be the case that another thread is about to insert or remove an element.
104 | *
105 | * @return the remaining capacity of this buffer
106 | */
107 | int remainingCapacity();
108 |
109 | /**
110 | * Removes all of the messages from this buffer.
111 | */
112 | void clear();
113 |
114 | /**
115 | * Can be used to set some exception originating from another thread, that should pop up using the buffer.
116 | *
117 | * @param exception the exception thrown from a thread that fills the buffer.
118 | */
119 | default void setException(RuntimeException exception) {
120 | logger.warn("setException was called, but is not implemented to do something with it", exception);
121 | }
122 | }
123 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/ConsumerPositionsUtil.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2023. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.apache.kafka.clients.consumer.Consumer;
20 | import org.apache.kafka.clients.consumer.OffsetAndTimestamp;
21 | import org.apache.kafka.common.TopicPartition;
22 | import org.axonframework.extensions.kafka.eventhandling.consumer.TopicSubscriber;
23 |
24 | import java.time.Instant;
25 | import java.util.HashMap;
26 | import java.util.List;
27 | import java.util.Map;
28 | import java.util.Optional;
29 | import javax.annotation.Nonnull;
30 |
31 | import static org.axonframework.extensions.kafka.eventhandling.consumer.ConsumerSeekUtil.topicPartitions;
32 |
33 | /**
34 | * Contains static util functions related to the Kafka consumer to find the correct positions.
35 | *
36 | * @author Gerard Klijs
37 | * @since 4.8.0
38 | */
39 | class ConsumerPositionsUtil {
40 |
41 | private ConsumerPositionsUtil() {
42 | //prevent instantiation
43 | }
44 |
45 | static Map getPositionsBasedOnTime(
46 | @Nonnull Consumer, ?> consumer,
47 | @Nonnull TopicSubscriber subscriber,
48 | @Nonnull Instant rawDefaultAt
49 | ) {
50 | List all = topicPartitions(consumer, subscriber);
51 | Map positions = new HashMap<>();
52 | OffsetSupplier offsetSupplier = new OffsetSupplier(consumer, rawDefaultAt, all);
53 | all.forEach(assignedPartition -> {
54 | Long offset = offsetSupplier.getOffset(assignedPartition);
55 | //if it's 0, we otherwise miss the first event
56 | if (offset > 1) {
57 | positions.put(assignedPartition, offset - 1);
58 | }
59 | });
60 | return positions;
61 | }
62 |
63 | static Map getHeadPositions(
64 | @Nonnull Consumer, ?> consumer,
65 | @Nonnull TopicSubscriber subscriber
66 | ) {
67 | List all = topicPartitions(consumer, subscriber);
68 | Map positions = new HashMap<>();
69 | Map endOffsets = consumer.endOffsets(all);
70 | endOffsets.forEach((assignedPartition, offset) -> {
71 | //if it's 0, we otherwise miss the first event
72 | if (offset > 1) {
73 | positions.put(assignedPartition, offset - 1);
74 | }
75 | });
76 | return positions;
77 | }
78 |
79 | private static class OffsetSupplier {
80 |
81 | private final Map partitionOffsetMap;
82 | private final Map endOffsets;
83 |
84 | private OffsetSupplier(Consumer, ?> consumer, Instant rawDefaultAt, List all) {
85 | long defaultAt = rawDefaultAt.toEpochMilli();
86 | Map timestampsToSearch = new HashMap<>();
87 | all.forEach(tp -> timestampsToSearch.put(tp, defaultAt));
88 | partitionOffsetMap = consumer.offsetsForTimes(timestampsToSearch);
89 | endOffsets = consumer.endOffsets(all);
90 | }
91 |
92 | private Optional getDefaultOffset(TopicPartition assignedPartition) {
93 | return Optional.ofNullable(partitionOffsetMap.get(assignedPartition))
94 | .map(OffsetAndTimestamp::offset);
95 | }
96 |
97 | private long getEndOffset(TopicPartition assignedPartition) {
98 | return endOffsets.get(assignedPartition);
99 | }
100 |
101 | private Long getOffset(TopicPartition assignedPartition) {
102 | return getDefaultOffset(assignedPartition).orElseGet(() -> getEndOffset(assignedPartition));
103 | }
104 | }
105 | }
106 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/KafkaMessageStream.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2019. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.axonframework.common.Registration;
20 | import org.axonframework.eventhandling.TrackedEventMessage;
21 | import org.axonframework.eventhandling.TrackingEventStream;
22 | import org.slf4j.Logger;
23 | import org.slf4j.LoggerFactory;
24 |
25 | import java.util.Optional;
26 | import java.util.concurrent.TimeUnit;
27 |
28 | import static org.axonframework.common.BuilderUtils.assertNonNull;
29 |
30 | /**
31 | * Create message stream from a specific Kafka topic. Messages are fetch in bulk and stored in an in-memory buffer. We
32 | * try to introduce some sort and stored them in a local buffer. Consumer position is tracked via {@link
33 | * KafkaTrackingToken}. Records are fetched from Kafka and stored in-memory buffer.
34 | *
35 | * This is not thread safe.
36 | *
37 | * @author Allard Buijze
38 | * @author Nakul Mishra
39 | * @since 4.0
40 | */
41 | public class KafkaMessageStream implements TrackingEventStream {
42 |
43 | private static final Logger logger = LoggerFactory.getLogger(KafkaMessageStream.class);
44 |
45 | private final Buffer buffer;
46 | private final Registration closeHandler;
47 | private KafkaEventMessage peekedEvent;
48 |
49 | /**
50 | * Create a {@link TrackingEventStream} dedicated to {@link KafkaEventMessage}s. Uses the provided {@code buffer} to
51 | * retrieve event messages from.
52 | *
53 | * @param buffer the {@link KafkaEventMessage} {@link Buffer} containing the fetched messages
54 | * @param closeHandler the service {@link Registration} which fills the buffer. Will be canceled upon executing a
55 | * {@link #close()}
56 | */
57 | @SuppressWarnings("WeakerAccess")
58 | public KafkaMessageStream(Buffer buffer, Registration closeHandler) {
59 | assertNonNull(buffer, "Buffer may not be null");
60 | this.buffer = buffer;
61 | this.closeHandler = closeHandler;
62 | }
63 |
64 | @Override
65 | public Optional> peek() {
66 | return Optional.ofNullable(
67 | peekedEvent == null && !hasNextAvailable(0, TimeUnit.NANOSECONDS) ? null : peekedEvent.value()
68 | );
69 | }
70 |
71 | @Override
72 | public boolean hasNextAvailable(int timeout, TimeUnit unit) {
73 | try {
74 | return peekedEvent != null || (peekedEvent = buffer.poll(timeout, unit)) != null;
75 | } catch (InterruptedException e) {
76 | logger.warn("Consumer thread was interrupted. Returning thread to event processor.", e);
77 | Thread.currentThread().interrupt();
78 | return false;
79 | }
80 | }
81 |
82 | @Override
83 | public TrackedEventMessage> nextAvailable() {
84 | try {
85 | return peekedEvent == null ? buffer.take().value() : peekedEvent.value();
86 | } catch (InterruptedException e) {
87 | logger.warn("Consumer thread was interrupted. Returning thread to event processor.", e);
88 | Thread.currentThread().interrupt();
89 | return null;
90 | } finally {
91 | peekedEvent = null;
92 | }
93 | }
94 |
95 | @Override
96 | public void close() {
97 | if (closeHandler != null) {
98 | closeHandler.close();
99 | }
100 | }
101 | }
102 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/KafkaRecordMetaData.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2019. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | /**
20 | * An interface for messages originating from Kafka capable of providing information about their source.
21 | *
22 | * @param the type of body used for this record
23 | * @author Nakul Mishra
24 | * @author Steven van Beelen
25 | * @since 4.0
26 | */
27 | public interface KafkaRecordMetaData {
28 |
29 | /**
30 | * The partition from which this record is received.
31 | *
32 | * @return an {@code int} defining the partition from which this record is received
33 | */
34 | int partition();
35 |
36 | /**
37 | * The position of the record in the corresponding Kafka {@code partition}.
38 | *
39 | * @return a {@code long} defining the position of the record in the corresponding Kafka {@code partition}
40 | */
41 | long offset();
42 |
43 | /**
44 | * The timestamp of the record.
45 | *
46 | * @return a {@code long} defining the timestamp of this record
47 | */
48 | long timestamp();
49 |
50 | /**
51 | * The value of this record.
52 | *
53 | * @return the value of this record of type {@code V}
54 | */
55 | V value();
56 | }
57 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/TopicPartitionDeserializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import com.fasterxml.jackson.databind.DeserializationContext;
20 | import com.fasterxml.jackson.databind.KeyDeserializer;
21 | import org.apache.kafka.common.TopicPartition;
22 | import org.axonframework.serialization.SerializationException;
23 |
24 | /**
25 | * Custom {@link KeyDeserializer} used to deserialize the {@link TopicPartition}.
26 | *
27 | * @author leechedan
28 | * @since 4.0
29 | */
30 | public class TopicPartitionDeserializer extends KeyDeserializer {
31 |
32 | private static final char HYPHEN = '-';
33 |
34 | @Override
35 | public TopicPartition deserializeKey(String key, DeserializationContext context) {
36 | if (null == key || key.lastIndexOf(HYPHEN) < 1) {
37 | return null;
38 | }
39 |
40 | int hyphenIndex = key.lastIndexOf(HYPHEN);
41 | String positionString = key.substring(hyphenIndex + 1);
42 |
43 | int position;
44 | try {
45 | position = Integer.parseInt(positionString);
46 | } catch (NumberFormatException e) {
47 | throw new SerializationException(String.format(
48 | "Cannot parse the position of TopicPartition from json:[%s].", key
49 | ));
50 | }
51 |
52 | if (position < 0) {
53 | throw new SerializationException("The position of the TopicPartition should be greater than zero.");
54 | }
55 |
56 | return new TopicPartition(key.substring(0, hyphenIndex), position);
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/TrackingRecordConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2019. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.apache.kafka.clients.consumer.ConsumerRecord;
20 | import org.apache.kafka.clients.consumer.ConsumerRecords;
21 | import org.axonframework.extensions.kafka.eventhandling.KafkaMessageConverter;
22 | import org.axonframework.extensions.kafka.eventhandling.consumer.RecordConverter;
23 | import org.slf4j.Logger;
24 | import org.slf4j.LoggerFactory;
25 |
26 | import java.util.ArrayList;
27 | import java.util.List;
28 |
29 | import static org.axonframework.common.Assert.nonNull;
30 |
31 | /**
32 | * {@link RecordConverter} instances which keeps track of the converted {@link ConsumerRecords} through a {@link
33 | * KafkaTrackingToken}. Consequently it converts the ConsumerRecords in to {@link KafkaEventMessage} instances.
34 | *
35 | * @param the key of the Kafka {@link ConsumerRecords} to be converted
36 | * @param the value type of Kafka {@link ConsumerRecords} to be converted
37 | * @author Steven van Beelen
38 | * @since 4.0
39 | */
40 | public class TrackingRecordConverter implements RecordConverter {
41 |
42 | private static final Logger logger = LoggerFactory.getLogger(TrackingRecordConverter.class);
43 |
44 | private final KafkaMessageConverter messageConverter;
45 | private KafkaTrackingToken currentToken;
46 |
47 | /**
48 | * Instantiates a {@link TrackingRecordConverter}, using the {@link KafkaMessageConverter} to convert {@link
49 | * ConsumerRecord} instances in to an {@link org.axonframework.eventhandling.EventMessage} instances. As it
50 | * traverses the {@link ConsumerRecords} it will advance the provided {@code token}. An {@link
51 | * IllegalArgumentException} will be thrown if the provided {@code token} is {@code null}.
52 | *
53 | * @param messageConverter the {@link KafkaMessageConverter} used to convert a {@link ConsumerRecord} in to an
54 | * {@link org.axonframework.eventhandling.EventMessage}
55 | * @param token the {@link KafkaTrackingToken} to advance for every fetched {@link ConsumerRecord}
56 | */
57 | public TrackingRecordConverter(KafkaMessageConverter messageConverter, KafkaTrackingToken token) {
58 | this.messageConverter = messageConverter;
59 | this.currentToken = nonNull(token, () -> "Token may not be null");
60 | }
61 |
62 | /**
63 | * {@inheritDoc}
64 | *
65 | * {@code E} is defined as a {@link KafkaEventMessage} for this implementation. Every {@link ConsumerRecord} will
66 | * advance the defined {@code token}'s position further with the ConsumerRecord's {@link ConsumerRecord#partition()}
67 | * and {@link ConsumerRecord#offset()}.
68 | */
69 | @Override
70 | public List convert(ConsumerRecords records) {
71 | List eventMessages = new ArrayList<>(records.count());
72 | for (ConsumerRecord consumerRecord : records) {
73 | messageConverter.readKafkaMessage(consumerRecord).ifPresent(eventMessage -> {
74 | KafkaTrackingToken nextToken =
75 | currentToken.advancedTo(consumerRecord.topic(),
76 | consumerRecord.partition(),
77 | consumerRecord.offset());
78 | logger.debug("Advancing token from [{}] to [{}]", currentToken, nextToken);
79 |
80 | currentToken = nextToken;
81 | eventMessages.add(KafkaEventMessage.from(eventMessage, consumerRecord, currentToken));
82 | });
83 | }
84 | return eventMessages;
85 | }
86 |
87 | /**
88 | * Return the current state of the {@link KafkaTrackingToken} this converter updates
89 | *
90 | * @return the current state of the {@link KafkaTrackingToken} this converter updates
91 | */
92 | public KafkaTrackingToken currentToken() {
93 | return currentToken;
94 | }
95 | }
96 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/TrackingTokenConsumerRebalanceListener.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.apache.kafka.clients.consumer.Consumer;
20 | import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
21 | import org.apache.kafka.common.TopicPartition;
22 | import org.axonframework.extensions.kafka.eventhandling.consumer.ConsumerSeekUtil;
23 | import org.slf4j.Logger;
24 | import org.slf4j.LoggerFactory;
25 |
26 | import java.lang.invoke.MethodHandles;
27 | import java.util.Collection;
28 | import java.util.List;
29 | import java.util.Map;
30 | import java.util.function.Supplier;
31 |
32 | /**
33 | * A {@link ConsumerRebalanceListener} which upon {@link ConsumerRebalanceListener#onPartitionsAssigned(Collection)}
34 | * will perform a {@link Consumer#seek(TopicPartition, long)} using the partition offsets in the given {@link
35 | * KafkaTrackingToken}.
36 | *
37 | * This implementation ensures that Axon is in charge of event handling progress by always starting from the beginning
38 | * of the stream for unknown partitions. This approach follows how a {@link org.axonframework.eventhandling.TrackingEventProcessor}
39 | * deals with unknown progress in any {@link org.axonframework.eventhandling.TrackingToken}.
40 | *
41 | * @param the key of the records the {@link Consumer} polls
42 | * @param the value type of the records the {@link Consumer} polls
43 | * @author Steven van Beelen
44 | * @since 4.0
45 | * @deprecated functionality moved to {@link ConsumerSeekUtil#seekToCurrentPositions(Consumer,
46 | * Supplier, List)} when group id was removed from the consumer.
47 | */
48 | @Deprecated
49 | @SuppressWarnings("squid:S1133") //removing would be a breaking change and can only be done in a major release
50 | public class TrackingTokenConsumerRebalanceListener implements ConsumerRebalanceListener {
51 |
52 | private static final Logger logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
53 |
54 | private final Consumer consumer;
55 | private final Supplier tokenSupplier;
56 |
57 | /**
58 | * Create a {@link ConsumerRebalanceListener} which uses a {@link KafkaTrackingToken} through the given {@code
59 | * tokenSupplier} to consciously {@link Consumer#seek(TopicPartition, long)} the offsets of the partitions assigned
60 | * to the given {@code consumer}.
61 | *
62 | * @param consumer the {@link Consumer} used to call {@link Consumer#seek(TopicPartition, long)} on
63 | * @param tokenSupplier the {@link Supplier} of a {@link KafkaTrackingToken}. This should provide the most recent
64 | * progress of the given {@code consumer}
65 | */
66 | public TrackingTokenConsumerRebalanceListener(Consumer consumer, Supplier tokenSupplier) {
67 | this.consumer = consumer;
68 | this.tokenSupplier = tokenSupplier;
69 | }
70 |
71 | @Override
72 | public void onPartitionsRevoked(Collection partitions) {
73 | // Not implemented
74 | }
75 |
76 | /**
77 | * {@inheritDoc}
78 | *
79 | * This implementation will use {@link Consumer#seek(TopicPartition, long)} for all given {@code assignedPartitions}
80 | * using the offsets known in the {@link KafkaTrackingToken}, retrieved through the {@code tokenSupplier}.
81 | *
82 | * If no offset is known for a given {@link TopicPartition} then we enforce the offset to {@code 0} to ensure all
83 | * known records for the new partition are processed. This could occur if polling is started for the first time or
84 | * if the number of partitions for the {@link Consumer}s topic is administratively adjusted.
85 | */
86 | @Override
87 | public void onPartitionsAssigned(Collection assignedPartitions) {
88 | KafkaTrackingToken currentToken = tokenSupplier.get();
89 | assignedPartitions.forEach(assignedPartition -> {
90 | Map tokenPartitionPositions = currentToken.getPositions();
91 |
92 | long offset = 0L;
93 | if (tokenPartitionPositions.containsKey(assignedPartition)) {
94 | offset = tokenPartitionPositions.get(assignedPartition) + 1;
95 | }
96 |
97 | logger.info("Seeking topic-partition [{}] with offset [{}]", assignedPartition, offset);
98 | consumer.seek(assignedPartition, offset);
99 | });
100 | }
101 | }
102 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/producer/ConfirmationMode.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.producer;
18 |
19 | /**
20 | * Modes for publishing Axon Event Messages to Kafka.
21 | *
22 | * - TRANSACTIONAL: use Kafka transactions while sending messages
23 | * - WAIT_FOR_ACK: send messages and wait for acknowledgment
24 | * - NONE: Fire and forget
25 | *
26 | *
27 | * @author Nakul Mishra
28 | * @since 4.0
29 | */
30 | public enum ConfirmationMode {
31 |
32 | /**
33 | * Indicates a confirmation mode which uses Kafka transactions whilst sending messages.
34 | */
35 | TRANSACTIONAL,
36 |
37 | /**
38 | * Indicates a confirmation mode which sends messages and waits for consumption acknowledgements.
39 | */
40 | WAIT_FOR_ACK,
41 |
42 | /**
43 | * Indicates a confirmation mode resembling fire and forget.
44 | */
45 | NONE;
46 |
47 | /**
48 | * Verify whether {@code this} confirmation mode is of type {@link #TRANSACTIONAL}.
49 | *
50 | * @return {@code true} if {@code this} confirmation mode matches {@link #TRANSACTIONAL}, {@code false} if it
51 | * doesn't
52 | */
53 | public boolean isTransactional() {
54 | return this == TRANSACTIONAL;
55 | }
56 |
57 | /**
58 | * Verify whether {@code this} confirmation mode is of type {@link #WAIT_FOR_ACK}.
59 | *
60 | * @return {@code true} if {@code this} confirmation mode matches {@link #WAIT_FOR_ACK}, {@code false} if it doesn't
61 | */
62 | public boolean isWaitForAck() {
63 | return this == WAIT_FOR_ACK;
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/producer/ProducerFactory.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.producer;
18 |
19 | import org.apache.kafka.clients.producer.Producer;
20 |
21 | /**
22 | * A functional interface towards building {@link Producer} instances.
23 | *
24 | * @param the key type of a build {@link Producer} instance
25 | * @param the value type of a build {@link Producer} instance
26 | * @author Nakul Mishra
27 | * @author Steven van Beelen
28 | * @since 4.0
29 | */
30 | public interface ProducerFactory {
31 |
32 | /**
33 | * Create a {@link Producer}.
34 | *
35 | * @return a {@link Producer}
36 | */
37 | Producer createProducer();
38 |
39 | /**
40 | * The {@link ConfirmationMode} all created {@link Producer} instances should comply to. Defaults to {@link
41 | * ConfirmationMode#NONE}.
42 | *
43 | * @return the configured confirmation mode
44 | */
45 | default ConfirmationMode confirmationMode() {
46 | return ConfirmationMode.NONE;
47 | }
48 |
49 | /**
50 | * Closes all {@link Producer} instances created by this factory.
51 | */
52 | void shutDown();
53 | }
54 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/producer/TopicResolver.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.producer;
18 |
19 | import org.axonframework.eventhandling.EventMessage;
20 |
21 | import java.util.Optional;
22 | import java.util.function.Function;
23 |
24 | /**
25 | * Interface to determine if an {@code EventMessage} should be published to Kafka, and if so to which topic. If the
26 | * result from the call is {@code Optional.empty()} is will not be published, else the result will be used for the
27 | * topic.
28 | *
29 | * @author Gerard Klijs
30 | * @since 4.6.0
31 | */
32 | @FunctionalInterface
33 | public interface TopicResolver extends Function, Optional> {
34 |
35 | /**
36 | * resolve an {@code EventMessage} to an optional topic to publish the event to
37 | *
38 | * @param event an {@code EventMessage}
39 | * @return the optional topic, when empty the event message will not be published
40 | */
41 | default Optional resolve(EventMessage> event) {
42 | return this.apply(event);
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/tokenstore/TokenStoreInitializationException.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.tokenstore;
18 |
19 | import org.axonframework.common.AxonException;
20 |
21 | /**
22 | * Exception thrown when failing to initialize the token store.
23 | *
24 | * @author Gerard Klijs
25 | * @since 4.6.0
26 | */
27 | public class TokenStoreInitializationException extends AxonException {
28 |
29 | /**
30 | * Initializes the exception using the given {@code message}.
31 | *
32 | * @param message The message describing the exception
33 | */
34 | public TokenStoreInitializationException(String message) {
35 | super(message);
36 | }
37 | }
38 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/tokenstore/TokenUpdateDeserializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.tokenstore;
18 |
19 | import org.apache.kafka.common.header.Headers;
20 | import org.apache.kafka.common.serialization.Deserializer;
21 |
22 | /**
23 | * Kafka deserializer used for the {@link KafkaTokenStore}.
24 | *
25 | * @author Gerard Klijs
26 | * @since 4.6.0
27 | */
28 | public class TokenUpdateDeserializer implements Deserializer {
29 |
30 | /**
31 | * This method should not be used, instead {@link #deserialize(String, Headers, byte[]) deserialize}, with headers
32 | * should be used.
33 | *
34 | * @param topic the topic the bytes are read from, part of the interface, currently only used for logging.
35 | * @param bytes the bytes received from the Kafka broker.
36 | * @return a {@link UnsupportedOperationException} exception
37 | */
38 | @Override
39 | public TokenUpdate deserialize(String topic, byte[] bytes) {
40 | throw new UnsupportedOperationException("deserialize should be called also using the headers");
41 | }
42 |
43 | /**
44 | * Deserializes the bytes to a {@link TokenUpdate} object
45 | *
46 | * @param topic the topic the bytes are read from, part of the interface, currently only used for logging.
47 | * @param headers the headers received from the Kafka broker.
48 | * @param data the bytes received from the Kafka broker.
49 | * @return a {@link TokenUpdate} object
50 | */
51 | @Override
52 | public TokenUpdate deserialize(String topic, Headers headers, byte[] data) {
53 | return new TokenUpdate(headers, data);
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/kafka/src/main/java/org/axonframework/extensions/kafka/eventhandling/tokenstore/TokenUpdateSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.tokenstore;
18 |
19 | import org.apache.kafka.common.header.Headers;
20 | import org.apache.kafka.common.serialization.Serializer;
21 |
22 | /**
23 | * Kafka serializer used for the {@link TokenUpdate}.
24 | *
25 | * @author Gerard Klijs
26 | * @since 4.6.0
27 | */
28 | public class TokenUpdateSerializer implements Serializer {
29 |
30 | /**
31 | * This method should not be used, instead {@link #serialize(String, Headers, TokenUpdate)}) serialize}, with
32 | * headers should be used.
33 | *
34 | * @param topic topic the bytes are written to, part of the interface, currently not used
35 | * @param tokenUpdate the token update object to send to Kafka
36 | * @return a {@link UnsupportedOperationException} exception
37 | */
38 | @Override
39 | @SuppressWarnings("squid:S1168") //needs to return null to work as tombstone
40 | public byte[] serialize(String topic, TokenUpdate tokenUpdate) {
41 | throw new UnsupportedOperationException("serialize should be called also using the headers");
42 | }
43 |
44 | /**
45 | * Serializes the {@code tokenUpdate} to bytes.
46 | *
47 | * @param topic topic the bytes are written to, part of the interface, currently not used
48 | * @param headers kafka headers
49 | * @param tokenUpdate the token update object to send to Kafka
50 | * @return the bytes to add to the Kafka record
51 | */
52 | @Override
53 | public byte[] serialize(String topic, Headers headers, TokenUpdate tokenUpdate) {
54 | tokenUpdate.setHeaders(headers);
55 | return tokenUpdate.getToken();
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/kafka/src/main/resources/META-INF/spring-devtools.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Copyright (c) 2010-2021. Axon Framework
3 | #
4 | # Licensed under the Apache License, Version 2.0 (the "License");
5 | # you may not use this file except in compliance with the License.
6 | # You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 | #
16 | restart.include.axon-kafka=axon-kafka-${project.version}.jar
17 | restart.include.axon-kafka-spring-boot-autoconfigure=axon-kafka-spring-boot-autoconfigure-${project.version}.jar
18 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/configuration/KafkaMessageSourceConfigurerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.configuration;
18 |
19 | import org.axonframework.config.Configuration;
20 | import org.axonframework.config.DefaultConfigurer;
21 | import org.axonframework.extensions.kafka.eventhandling.consumer.subscribable.SubscribableKafkaMessageSource;
22 | import org.junit.jupiter.api.*;
23 | import org.junit.jupiter.api.extension.*;
24 | import org.mockito.*;
25 | import org.mockito.junit.jupiter.*;
26 |
27 | import static org.mockito.Mockito.*;
28 |
29 | /**
30 | * Test classes verifying registered message sources are started and closed through the {@link
31 | * org.axonframework.config.ModuleConfiguration} API.
32 | *
33 | * @author Steven van Beelen
34 | */
35 | @ExtendWith(MockitoExtension.class)
36 | class KafkaMessageSourceConfigurerTest {
37 |
38 | private Configuration configuration;
39 |
40 | private final KafkaMessageSourceConfigurer testSubject = new KafkaMessageSourceConfigurer();
41 |
42 | @BeforeEach
43 | void setUp() {
44 | configuration = DefaultConfigurer.defaultConfiguration()
45 | .buildConfiguration();
46 | }
47 |
48 | @Test
49 | void testStartInitiatesRegisteredSubscribableSources(
50 | @Mock SubscribableKafkaMessageSource, ?> sourceOne,
51 | @Mock SubscribableKafkaMessageSource, ?> sourceTwo
52 | ) {
53 | testSubject.configureSubscribableSource(conf -> sourceOne);
54 | testSubject.configureSubscribableSource(conf -> sourceTwo);
55 |
56 | testSubject.initialize(configuration);
57 | configuration.start();
58 |
59 | verify(sourceOne).start();
60 | verify(sourceTwo).start();
61 | }
62 |
63 | @Test
64 | void testShutdownClosesRegisteredSubscribableSources(
65 | @Mock SubscribableKafkaMessageSource, ?> sourceOne,
66 | @Mock SubscribableKafkaMessageSource, ?> sourceTwo
67 | ) {
68 | testSubject.configureSubscribableSource(conf -> sourceOne);
69 | testSubject.configureSubscribableSource(conf -> sourceTwo);
70 |
71 | testSubject.initialize(configuration);
72 | configuration.shutdown();
73 |
74 | verify(sourceOne).close();
75 | verify(sourceTwo).close();
76 | }
77 | }
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/benchmark/SimpleRandom.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2023. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.benchmark;
18 |
19 | import java.util.concurrent.atomic.AtomicLong;
20 |
21 | /**
22 | * @author Pavel Tcholakov.
23 | * @see JCTools
24 | */
25 | final class SimpleRandom {
26 |
27 | private final static long multiplier = 0x5DEECE66DL;
28 | private final static long addend = 0xBL;
29 | private final static long mask = (1L << 48) - 1;
30 | private static final AtomicLong seq = new AtomicLong(-715159705);
31 | private long seed;
32 |
33 | SimpleRandom() {
34 | seed = System.nanoTime() + seq.getAndAdd(129);
35 | }
36 |
37 | public int next() {
38 | long nextSeed = (seed * multiplier + addend) & mask;
39 | seed = nextSeed;
40 | return ((int) (nextSeed >>> 17)) & 0x7FFFFFFF;
41 | }
42 | }
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/consumer/ConsumerRecordConverter.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2019. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer;
18 |
19 | import org.apache.kafka.clients.consumer.ConsumerRecord;
20 | import org.apache.kafka.clients.producer.ProducerRecord;
21 | import org.axonframework.eventhandling.EventMessage;
22 | import org.axonframework.extensions.kafka.eventhandling.KafkaMessageConverter;
23 |
24 | import java.util.Optional;
25 |
26 | import static org.axonframework.eventhandling.GenericEventMessage.asEventMessage;
27 |
28 | /**
29 | * A {@link KafkaMessageConverter} implementation solely intended to test message consumption.
30 | *
31 | * @author Steven van Beelen
32 | */
33 | class ConsumerRecordConverter implements KafkaMessageConverter {
34 |
35 | @Override
36 | public ProducerRecord createKafkaMessage(EventMessage> eventMessage, String topic) {
37 | throw new UnsupportedOperationException();
38 | }
39 |
40 | @Override
41 | public Optional> readKafkaMessage(ConsumerRecord consumerRecord) {
42 | return Optional.of(asEventMessage(consumerRecord.value()));
43 | }
44 | }
45 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/consumer/DefaultConsumerFactoryIntegrationTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer;
18 |
19 | import org.apache.kafka.clients.consumer.Consumer;
20 | import org.apache.kafka.clients.producer.Producer;
21 | import org.apache.kafka.clients.producer.ProducerRecord;
22 | import org.axonframework.common.AxonConfigurationException;
23 | import org.axonframework.extensions.kafka.eventhandling.producer.ProducerFactory;
24 | import org.axonframework.extensions.kafka.eventhandling.util.KafkaAdminUtils;
25 | import org.axonframework.extensions.kafka.eventhandling.util.KafkaContainerTest;
26 | import org.junit.jupiter.api.*;
27 |
28 | import java.util.Collections;
29 |
30 | import static org.axonframework.extensions.kafka.eventhandling.util.ConsumerConfigUtil.DEFAULT_GROUP_ID;
31 | import static org.axonframework.extensions.kafka.eventhandling.util.ConsumerConfigUtil.minimal;
32 | import static org.axonframework.extensions.kafka.eventhandling.util.KafkaTestUtils.getRecords;
33 | import static org.axonframework.extensions.kafka.eventhandling.util.ProducerConfigUtil.producerFactory;
34 | import static org.junit.jupiter.api.Assertions.*;
35 | import static org.mockito.Mockito.*;
36 |
37 | /**
38 | * Tests for the {@link DefaultConsumerFactory}, asserting construction and utilization of the class.
39 | *
40 | * @author Nakul Mishra
41 | * @author Steven van Beelen
42 | */
43 | class DefaultConsumerFactoryIntegrationTest extends KafkaContainerTest {
44 |
45 | private static final String TEST_TOPIC = "testCreatedConsumer_ValidConfig_CanCommunicateToKafka";
46 |
47 | private ProducerFactory producerFactory;
48 | private Consumer, ?> testConsumer;
49 |
50 | @BeforeAll
51 | static void before() {
52 | KafkaAdminUtils.createTopics(getBootstrapServers(), TEST_TOPIC);
53 | }
54 |
55 | @AfterAll
56 | public static void after() {
57 | KafkaAdminUtils.deleteTopics(getBootstrapServers(), TEST_TOPIC);
58 | }
59 |
60 | @BeforeEach
61 | void setUp() {
62 | producerFactory = producerFactory(getBootstrapServers());
63 | testConsumer = mock(Consumer.class);
64 | }
65 |
66 | @AfterEach
67 | void tearDown() {
68 | producerFactory.shutDown();
69 | testConsumer.close();
70 | }
71 |
72 | @Test
73 | void testCreateConsumerInvalidConfig() {
74 | assertThrows(AxonConfigurationException.class, () -> new DefaultConsumerFactory<>(null));
75 | }
76 |
77 | @Test
78 | void testCreatedConsumerValidConfigCanCommunicateToKafka() {
79 | String testTopic = "testCreatedConsumer_ValidConfig_CanCommunicateToKafka";
80 |
81 | Producer testProducer = producerFactory.createProducer();
82 | testProducer.send(new ProducerRecord<>(testTopic, 0, null, null, "foo"));
83 | testProducer.flush();
84 |
85 | ConsumerFactory, ?> testSubject = new DefaultConsumerFactory<>(minimal(KAFKA_CONTAINER
86 | .getBootstrapServers()));
87 | testConsumer = testSubject.createConsumer(DEFAULT_GROUP_ID);
88 | testConsumer.subscribe(Collections.singleton(testTopic));
89 |
90 | assertEquals(1, getRecords(testConsumer).count());
91 | }
92 | }
93 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/KafkaTrackingTokenSerializationTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.apache.kafka.common.TopicPartition;
20 | import org.axonframework.eventhandling.ReplayToken;
21 | import org.axonframework.eventhandling.TrackingToken;
22 | import org.axonframework.extensions.kafka.utils.TestSerializer;
23 | import org.junit.*;
24 | import org.junit.runner.*;
25 | import org.junit.runners.*;
26 |
27 | import java.util.Collection;
28 |
29 | import static java.util.Collections.singletonMap;
30 | import static org.axonframework.extensions.kafka.eventhandling.consumer.streamable.KafkaTrackingToken.newInstance;
31 | import static org.junit.jupiter.api.Assertions.*;
32 |
33 | /**
34 | * Test class validating the {@link KafkaTrackingToken} can be serialized and deserialized with a {@link
35 | * org.axonframework.serialization.Serializer}.
36 | *
37 | * @author leechedan
38 | */
39 | @RunWith(Parameterized.class)
40 | public class KafkaTrackingTokenSerializationTest {
41 |
42 | private static final String TEST_TOPIC = "topic";
43 | private static final int TEST_PARTITION = 0;
44 | private static final TopicPartition TEST_TOPIC_PARTITION = new TopicPartition(TEST_TOPIC, TEST_PARTITION);
45 |
46 | private final TestSerializer serializer;
47 |
48 | public KafkaTrackingTokenSerializationTest(TestSerializer serializer) {
49 | this.serializer = serializer;
50 | }
51 |
52 | @Parameterized.Parameters(name = "{index} {0}")
53 | public static Collection serializers() {
54 | return TestSerializer.all();
55 | }
56 |
57 | @Test
58 | public void testReplayTokenShouldBeSerializable() {
59 | ReplayToken tokenReset = new ReplayToken(nonEmptyToken(TEST_TOPIC_PARTITION, 0L));
60 | KafkaTrackingToken tokenStart = nonEmptyToken(TEST_TOPIC_PARTITION, 1L);
61 | ReplayToken replayToken = new ReplayToken(tokenReset, tokenStart);
62 | String serializeReplayToken = serializer.serialize(replayToken);
63 | TrackingToken deserializeReplayToken = serializer.deserialize(serializeReplayToken, ReplayToken.class);
64 | assertEquals(deserializeReplayToken, replayToken);
65 | }
66 |
67 | @Test
68 | public void testTokenShouldBeSerializable() {
69 | KafkaTrackingToken token = nonEmptyToken(TEST_TOPIC_PARTITION, 0L);
70 | String serializeCopy = serializer.serialize(token);
71 | KafkaTrackingToken deserializeCopy = serializer.deserialize(serializeCopy, KafkaTrackingToken.class);
72 | assertEquals(deserializeCopy, token);
73 | }
74 |
75 | @SuppressWarnings("SameParameterValue")
76 | private static KafkaTrackingToken nonEmptyToken(TopicPartition topic, Long pos) {
77 | return newInstance(singletonMap(topic, pos));
78 | }
79 | }
80 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/consumer/streamable/TopicPartitionDeserializerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.consumer.streamable;
18 |
19 | import org.apache.kafka.common.TopicPartition;
20 | import org.junit.*;
21 |
22 | import java.util.Arrays;
23 | import java.util.List;
24 |
25 | import static org.junit.jupiter.api.Assertions.*;
26 |
27 | /**
28 | * Tests for the {@link TopicPartitionDeserializer}.
29 | *
30 | * @author leechedan
31 | */
32 | public class TopicPartitionDeserializerTest {
33 |
34 | private static final List TOPIC_PARTITIONS = Arrays.asList(
35 | new TopicPartition("local", 0),
36 | new TopicPartition("local-", 1),
37 | new TopicPartition("local-event", 100)
38 | );
39 |
40 | private final TopicPartitionDeserializer testSubject = new TopicPartitionDeserializer();
41 |
42 | @Test
43 | public void testDeserializeShouldSuccess() {
44 | TOPIC_PARTITIONS.forEach(
45 | item -> assertEquals(item, testSubject.deserializeKey(item.toString(), null), item + " fail")
46 | );
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/producer/KafkaPublisherBuilderTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2023. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.producer;
18 |
19 | import org.junit.jupiter.api.*;
20 | import org.junit.jupiter.api.extension.*;
21 | import org.mockito.*;
22 | import org.mockito.junit.jupiter.*;
23 |
24 | import static org.junit.jupiter.api.Assertions.*;
25 |
26 | @ExtendWith(MockitoExtension.class)
27 | class KafkaPublisherBuilderTest {
28 |
29 | @Mock
30 | private ProducerFactory producerFactory;
31 |
32 | @Test
33 | void testKafkaPublisherInitialisationShouldNotThrowException() {
34 | assertDoesNotThrow(() -> KafkaPublisher.builder()
35 | .producerFactory(producerFactory)
36 | .build());
37 | }
38 | }
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/tokenstore/TokenUpdateDeserializerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.tokenstore;
18 |
19 | import org.apache.kafka.common.header.Headers;
20 | import org.apache.kafka.common.header.internals.RecordHeaders;
21 | import org.axonframework.extensions.kafka.eventhandling.HeaderUtils;
22 | import org.junit.jupiter.api.*;
23 |
24 | import java.time.Instant;
25 | import java.util.UUID;
26 |
27 | import static org.junit.jupiter.api.Assertions.*;
28 |
29 | /**
30 | * Test class validating the {@link TokenUpdateSerializer}.
31 | *
32 | * @author Gerard Klijs
33 | */
34 | class TokenUpdateDeserializerTest {
35 |
36 | @Test
37 | void testDeserializeUnsupportedOperation() {
38 | TokenUpdateDeserializer deserializer = new TokenUpdateDeserializer();
39 | byte[] bytes = new byte[0];
40 | assertThrows(UnsupportedOperationException.class, () -> deserializer.deserialize("topic", bytes));
41 | }
42 |
43 | @Test
44 | void testDeserializerMostlyEmpty() {
45 | TokenUpdateDeserializer deserializer = new TokenUpdateDeserializer();
46 | byte[] bytes = new byte[0];
47 | Headers headers = new RecordHeaders();
48 | HeaderUtils.addHeader(headers, "id", UUID.randomUUID());
49 | Instant now = Instant.now();
50 | HeaderUtils.addHeader(headers, "timestamp", now.toEpochMilli());
51 | TokenUpdate update = deserializer.deserialize("topic", headers, bytes);
52 | assertNotNull(update);
53 | assertEquals(now.toEpochMilli(), update.getTimestamp().toEpochMilli());
54 | }
55 | }
56 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/tokenstore/TokenUpdateSerializerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2022. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.tokenstore;
18 |
19 | import org.apache.kafka.common.header.Headers;
20 | import org.apache.kafka.common.header.internals.RecordHeaders;
21 | import org.axonframework.eventhandling.GlobalSequenceTrackingToken;
22 | import org.axonframework.eventhandling.TrackingToken;
23 | import org.axonframework.eventhandling.tokenstore.AbstractTokenEntry;
24 | import org.axonframework.eventhandling.tokenstore.GenericTokenEntry;
25 | import org.axonframework.extensions.kafka.eventhandling.HeaderUtils;
26 | import org.axonframework.serialization.json.JacksonSerializer;
27 | import org.junit.jupiter.api.*;
28 | import org.junit.jupiter.api.extension.*;
29 | import org.mockito.junit.jupiter.*;
30 |
31 | import static org.junit.jupiter.api.Assertions.*;
32 |
33 | /**
34 | * Test class validating the {@link TokenUpdateDeserializer}.
35 | *
36 | * @author Gerard Klijs
37 | */
38 | @ExtendWith(MockitoExtension.class)
39 | class TokenUpdateSerializerTest {
40 |
41 | @Test
42 | void testSerializeUnsupportedOperation() {
43 | TokenUpdateSerializer serializer = new TokenUpdateSerializer();
44 | assertThrows(UnsupportedOperationException.class, () -> serializer.serialize("topic", null));
45 | }
46 |
47 | @Test
48 | void testSerializeHappyFlow() {
49 | TokenUpdateSerializer serializer = new TokenUpdateSerializer();
50 | TrackingToken someToken = new GlobalSequenceTrackingToken(42);
51 | AbstractTokenEntry tokenEntry =
52 | new GenericTokenEntry<>(someToken,
53 | JacksonSerializer.defaultSerializer(),
54 | byte[].class,
55 | "processorName",
56 | 0);
57 | TokenUpdate update = new TokenUpdate(tokenEntry, 0);
58 | Headers headers = new RecordHeaders();
59 | byte[] bytes = serializer.serialize("topic", headers, update);
60 | assertNotNull(bytes);
61 | assertNotEquals(0, bytes.length);
62 | assertEquals(GlobalSequenceTrackingToken.class.getCanonicalName(),
63 | HeaderUtils.valueAsString(headers, "tokenType"));
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/util/AssertUtils.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2019. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.util;
18 |
19 | import java.time.Duration;
20 |
21 | /**
22 | * Utility class for special assertions.
23 | */
24 | public class AssertUtils {
25 |
26 | private AssertUtils() {
27 | // Utility class
28 | }
29 |
30 | /**
31 | * Assert that the given {@code assertion} succeeds with the given {@code time} and {@code unit}.
32 | *
33 | * @param duration The time in which the assertion must pass
34 | * @param assertion the assertion to succeed within the deadline
35 | */
36 | public static void assertWithin(Duration duration, Runnable assertion) {
37 | long now = System.currentTimeMillis();
38 | long deadline = now + duration.toMillis();
39 | do {
40 | try {
41 | assertion.run();
42 | break;
43 | } catch (AssertionError e) {
44 | Thread.yield();
45 | if (now >= deadline) {
46 | throw e;
47 | }
48 | }
49 | now = System.currentTimeMillis();
50 | } while (true);
51 | }
52 | }
53 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/util/HeaderAssertUtil.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2018. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.util;
18 |
19 | import org.apache.kafka.common.header.Headers;
20 | import org.axonframework.eventhandling.DomainEventMessage;
21 | import org.axonframework.eventhandling.EventMessage;
22 | import org.axonframework.serialization.SerializedObject;
23 |
24 | import static org.axonframework.extensions.kafka.eventhandling.HeaderUtils.*;
25 | import static org.axonframework.messaging.Headers.*;
26 | import static org.junit.jupiter.api.Assertions.*;
27 |
28 | /**
29 | * Utility for asserting Kafka headers sent via Axon.
30 | *
31 | * @author Nakul Mishra
32 | * @author Steven van Beelen
33 | */
34 | public abstract class HeaderAssertUtil {
35 |
36 | private HeaderAssertUtil() {
37 | // Utility class
38 | }
39 |
40 | public static void assertEventHeaders(String metaDataKey,
41 | EventMessage> eventMessage,
42 | SerializedObject so,
43 | Headers headers) {
44 | assertTrue(headers.toArray().length >= 5);
45 | assertEquals(eventMessage.getIdentifier(), valueAsString(headers, MESSAGE_ID));
46 | assertEquals(eventMessage.getTimestamp().toEpochMilli(), valueAsLong(headers, MESSAGE_TIMESTAMP));
47 | assertEquals(so.getType().getName(), valueAsString(headers, MESSAGE_TYPE));
48 | assertEquals(so.getType().getRevision(), valueAsString(headers, MESSAGE_REVISION));
49 | assertEquals(eventMessage.getMetaData().get(metaDataKey),
50 | valueAsString(headers, generateMetadataKey(metaDataKey)));
51 | }
52 |
53 | public static void assertDomainHeaders(DomainEventMessage> eventMessage, Headers headers) {
54 | assertTrue(headers.toArray().length >= 8);
55 | assertEquals(eventMessage.getSequenceNumber(), valueAsLong(headers, AGGREGATE_SEQ));
56 | assertEquals(eventMessage.getAggregateIdentifier(), valueAsString(headers, AGGREGATE_ID));
57 | assertEquals(eventMessage.getType(), valueAsString(headers, AGGREGATE_TYPE));
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/util/KafkaContainerClusterTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2023. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.util;
18 |
19 | import org.testcontainers.junit.jupiter.Container;
20 | import org.testcontainers.junit.jupiter.Testcontainers;
21 |
22 | /**
23 | * A {@link KafkaContainerCluster} set up by using {@link Testcontainers}.
24 | *
25 | * @author Lucas Campos
26 | */
27 | @Testcontainers
28 | public abstract class KafkaContainerClusterTest {
29 |
30 | @Container
31 | protected static final KafkaContainerCluster KAFKA_CLUSTER = new KafkaContainerCluster("5.4.3", 3, 1);
32 |
33 | protected static String getBootstrapServers() {
34 | return KAFKA_CLUSTER.getBootstrapServers();
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/eventhandling/util/KafkaContainerTest.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.eventhandling.util;
18 |
19 | import org.testcontainers.containers.KafkaContainer;
20 | import org.testcontainers.junit.jupiter.Container;
21 | import org.testcontainers.junit.jupiter.Testcontainers;
22 | import org.testcontainers.utility.DockerImageName;
23 |
24 | /**
25 | * A single {@link KafkaContainer} set up by using {@link Testcontainers}.
26 | *
27 | * @author Lucas Campos
28 | */
29 | @Testcontainers
30 | public abstract class KafkaContainerTest {
31 |
32 | @Container
33 | protected static final KafkaContainer KAFKA_CONTAINER =
34 | new KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka"));
35 |
36 | protected static String getBootstrapServers() {
37 | return KAFKA_CONTAINER.getBootstrapServers();
38 | }
39 | }
40 |
--------------------------------------------------------------------------------
/kafka/src/test/java/org/axonframework/extensions/kafka/utils/TestSerializer.java:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright (c) 2010-2021. Axon Framework
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License");
5 | * you may not use this file except in compliance with the License.
6 | * You may obtain a copy of the License at
7 | *
8 | * http://www.apache.org/licenses/LICENSE-2.0
9 | *
10 | * Unless required by applicable law or agreed to in writing, software
11 | * distributed under the License is distributed on an "AS IS" BASIS,
12 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | * See the License for the specific language governing permissions and
14 | * limitations under the License.
15 | */
16 |
17 | package org.axonframework.extensions.kafka.utils;
18 |
19 | import com.thoughtworks.xstream.XStream;
20 | import org.axonframework.serialization.JavaSerializer;
21 | import org.axonframework.serialization.SerializedObject;
22 | import org.axonframework.serialization.Serializer;
23 | import org.axonframework.serialization.SimpleSerializedObject;
24 | import org.axonframework.serialization.SimpleSerializedType;
25 | import org.axonframework.serialization.json.JacksonSerializer;
26 | import org.axonframework.serialization.xml.CompactDriver;
27 | import org.axonframework.serialization.xml.XStreamSerializer;
28 | import org.junit.*;
29 |
30 | import java.util.Base64;
31 | import java.util.Collection;
32 | import java.util.EnumSet;
33 |
34 | /**
35 | * Enumeration of serializers for testing purposes.
36 | *
37 | * @author JohT
38 | */
39 | @Ignore
40 | public enum TestSerializer {
41 |
42 | JAVA {
43 | @SuppressWarnings("deprecation")
44 | private final Serializer serializer = JavaSerializer.builder().build();
45 |
46 | @Override
47 | public Serializer getSerializer() {
48 | return serializer;
49 | }
50 |
51 | @Override
52 | public String serialize(Object object) {
53 | return Base64.getEncoder().encodeToString(getSerializer().serialize(object, byte[].class).getData());
54 | }
55 |
56 | @Override
57 | public T deserialize(String serialized, Class type) {
58 | return getSerializer().deserialize(asSerializedData(Base64.getDecoder().decode(serialized), type));
59 | }
60 | },
61 | XSTREAM {
62 | private final Serializer serializer = createSerializer();
63 |
64 | private XStreamSerializer createSerializer() {
65 | XStream xStream = new XStream(new CompactDriver());
66 | xStream.allowTypesByWildcard(new String[]{"org.apache.kafka.**"});
67 | return XStreamSerializer.builder()
68 | .xStream(xStream)
69 | .classLoader(this.getClass().getClassLoader())
70 | .build();
71 | }
72 |
73 | @Override
74 | public Serializer getSerializer() {
75 | return serializer;
76 | }
77 | },
78 | JACKSON {
79 | private final Serializer serializer = JacksonSerializer.builder().build();
80 |
81 | @Override
82 | public Serializer getSerializer() {
83 | return serializer;
84 | }
85 | };
86 |
87 | public String serialize(Object object) {
88 | return new String(getSerializer().serialize(object, byte[].class).getData());
89 | }
90 |
91 | public T deserialize(String serialized, Class type) {
92 | return getSerializer().deserialize(asSerializedData(serialized.getBytes(), type));
93 | }
94 |
95 | public abstract Serializer getSerializer();
96 |
97 | public static Collection all() {
98 | return EnumSet.allOf(TestSerializer.class);
99 | }
100 |
101 | static SerializedObject asSerializedData(byte[] serialized, Class type) {
102 | SimpleSerializedType serializedType = new SimpleSerializedType(type.getName(), null);
103 | return new SimpleSerializedObject<>(serialized, byte[].class, serializedType);
104 | }
105 | }
106 |
--------------------------------------------------------------------------------
/kafka/src/test/resources/log4j2.properties:
--------------------------------------------------------------------------------
1 | #
2 | # Copyright (c) 2010-2020. Axon Framework
3 | #
4 | # Licensed under the Apache License, Version 2.0 (the "License");
5 | # you may not use this file except in compliance with the License.
6 | # You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 | #
16 | name=AxonTestConfiguration
17 | appenders = console
18 |
19 | appender.console.type = Console
20 | appender.console.name = STDOUT
21 | appender.console.layout.type = PatternLayout
22 | appender.console.layout.pattern = %d [%t] %-5p %-30.30c{1} %x - %m%n
23 |
24 | rootLogger.level = warn
25 | rootLogger.appenderRefs = stdout
26 | rootLogger.appenderRef.stdout.ref = STDOUT
27 |
28 | logger.axon.name = org.axonframework
29 | logger.axon.level = info
30 | logger.axon.additivity = false
31 | logger.axon.appenderRefs = stdout
32 | logger.axon.appenderRef.stdout.ref = STDOUT
33 |
34 | logger.chaining-converter.name = org.axonframework.serialization.ChainingConverter
35 | logger.chaining-converter.level = OFF
36 |
37 | logger.kafka.name = org.apache.kafka
38 | logger.kafka.level = off
39 |
40 | logger.kafka-clients.name = org.apache.kafka.clients
41 | logger.kafka-clients.level = off
42 |
43 | logger.kafka-server.name = kafka.server
44 | logger.kafka-server.level = off
45 |
46 | logger.kafka-common.name = kafka.common
47 | logger.kafka-common.level = off
48 |
49 | logger.kafka-spring.name = org.springframework.kafka
50 | logger.kafka-spring.level = off
51 |
52 | logger.zookeeper.name = org.apache.zookeeper
53 | logger.zookeeper.level = off
54 |
55 | logger.metrics-reporting.name = com.yammer.metrics.reporting
56 | logger.metrics-reporting.level = off
57 |
--------------------------------------------------------------------------------