-
Notifications
You must be signed in to change notification settings - Fork 210
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
sqs source: json codec support to split sqs message into multiple events #5330
base: main
Are you sure you want to change the base?
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* Copyright OpenSearch Contributors | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.source.sqs; | ||
|
||
import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; | ||
import org.opensearch.dataprepper.model.buffer.Buffer; | ||
import org.opensearch.dataprepper.model.codec.InputCodec; | ||
import org.opensearch.dataprepper.model.event.Event; | ||
import org.opensearch.dataprepper.model.event.EventMetadata; | ||
import org.opensearch.dataprepper.model.record.Record; | ||
import org.slf4j.Logger; | ||
import org.slf4j.LoggerFactory; | ||
import software.amazon.awssdk.services.sqs.model.Message; | ||
import software.amazon.awssdk.services.sqs.model.MessageAttributeValue; | ||
import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; | ||
|
||
import java.io.ByteArrayInputStream; | ||
import java.nio.charset.StandardCharsets; | ||
import java.util.Map; | ||
import java.util.concurrent.TimeoutException; | ||
|
||
public class BulkSqsMessageHandler implements SqsMessageHandler { | ||
private static final Logger LOG = LoggerFactory.getLogger(BulkSqsMessageHandler.class); | ||
private final InputCodec codec; | ||
|
||
public BulkSqsMessageHandler(final InputCodec codec) { | ||
this.codec = codec; | ||
} | ||
|
||
@Override | ||
public void handleMessage(final Message message, | ||
final String url, | ||
final Buffer<Record<Event>> buffer, | ||
final int bufferTimeoutMillis, | ||
final AcknowledgementSet acknowledgementSet) { | ||
try { | ||
final String sqsBody = message.body(); | ||
ByteArrayInputStream inputStream = new ByteArrayInputStream(sqsBody.getBytes(StandardCharsets.UTF_8)); | ||
codec.parse(inputStream, record -> { | ||
final Event event = record.getData(); | ||
final EventMetadata eventMetadata = event.getMetadata(); | ||
eventMetadata.setAttribute("queueUrl", url); | ||
for (Map.Entry<MessageSystemAttributeName, String> entry : message.attributes().entrySet()) { | ||
final String originalKey = entry.getKey().toString(); | ||
final String lowerCamelCaseKey = originalKey.substring(0, 1).toLowerCase() + originalKey.substring(1);; | ||
eventMetadata.setAttribute(lowerCamelCaseKey, entry.getValue()); | ||
} | ||
|
||
for (Map.Entry<String, MessageAttributeValue> entry : message.messageAttributes().entrySet()) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A lot of this code is similar to the I think the best option is to update the existing It might look like:
This has some advantages of being able to use There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I removed the BulkMessageHandler and added strategies instead. buffer.writeAll is also used instead of buffer.write now |
||
final String originalKey = entry.getKey(); | ||
final String lowerCamelCaseKey = originalKey.substring(0, 1).toLowerCase() + originalKey.substring(1);; | ||
eventMetadata.setAttribute(lowerCamelCaseKey, entry.getValue().stringValue()); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. A lot of this processing can be shared, which has the added benefit of avoiding extra memory. You can do this in the refactoring that I suggest above. That is, before the loop over the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. great idea, the metadata/attributes of a bulk message would remain the same for every event in that message |
||
} | ||
|
||
if (acknowledgementSet != null) { | ||
acknowledgementSet.add(event); | ||
} | ||
|
||
try { | ||
buffer.write(record, bufferTimeoutMillis); | ||
} catch (TimeoutException e) { | ||
throw new RuntimeException(e); | ||
} | ||
}); | ||
} catch (final Exception e) { | ||
LOG.error("Error processing SQS message: {}", e.getMessage(), e); | ||
throw new RuntimeException(e); | ||
} | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,72 @@ | ||
/* | ||
* SPDX-License-Identifier: Apache-2.0 | ||
*/ | ||
|
||
package org.opensearch.dataprepper.plugins.source.sqs; | ||
|
||
import org.junit.jupiter.api.BeforeEach; | ||
import org.junit.jupiter.api.Test; | ||
import org.mockito.ArgumentCaptor; | ||
import org.opensearch.dataprepper.model.buffer.Buffer; | ||
import org.opensearch.dataprepper.model.codec.InputCodec; | ||
import org.opensearch.dataprepper.model.event.Event; | ||
import org.opensearch.dataprepper.model.event.EventMetadata; | ||
import org.opensearch.dataprepper.model.record.Record; | ||
import software.amazon.awssdk.services.sqs.model.Message; | ||
|
||
import java.io.InputStream; | ||
import java.util.function.Consumer; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.mockito.ArgumentMatchers.any; | ||
import static org.mockito.ArgumentMatchers.eq; | ||
import static org.mockito.Mockito.doAnswer; | ||
import static org.mockito.Mockito.mock; | ||
import static org.mockito.Mockito.times; | ||
import static org.mockito.Mockito.verify; | ||
import static org.mockito.Mockito.when; | ||
|
||
class BulkSqsMessageHandlerTest { | ||
|
||
private InputCodec mockCodec; | ||
private Buffer<Record<Event>> mockBuffer; | ||
private BulkSqsMessageHandler bulkSqsMessageHandler; | ||
private int bufferTimeoutMillis; | ||
|
||
@BeforeEach | ||
void setUp() { | ||
mockCodec = mock(InputCodec.class); | ||
mockBuffer = mock(Buffer.class); | ||
bulkSqsMessageHandler = new BulkSqsMessageHandler(mockCodec); | ||
bufferTimeoutMillis = 10000; | ||
} | ||
|
||
@Test | ||
void handleMessage_callsBufferWriteOnce() throws Exception { | ||
final Message message = Message.builder() | ||
.body("{\"someKey\":\"someValue\"}") | ||
.build(); | ||
final String queueUrl = "https://sqs.us-east-1.amazonaws.com/123456789012/test-queue"; | ||
|
||
doAnswer(invocation -> { | ||
@SuppressWarnings("unchecked") | ||
Consumer<Record<Event>> eventConsumer = invocation.getArgument(1); | ||
final Event mockEvent = mock(Event.class); | ||
final EventMetadata mockMetadata = mock(EventMetadata.class); | ||
when(mockEvent.getMetadata()).thenReturn(mockMetadata); | ||
when(mockMetadata.getEventType()).thenReturn("DOCUMENT"); | ||
eventConsumer.accept(new Record<>(mockEvent)); | ||
return null; | ||
}).when(mockCodec).parse(any(InputStream.class), any()); | ||
|
||
bulkSqsMessageHandler.handleMessage(message, queueUrl, mockBuffer, bufferTimeoutMillis, null); | ||
ArgumentCaptor<Record<Event>> argumentCaptor = ArgumentCaptor.forClass(Record.class); | ||
verify(mockBuffer, times(1)).write(argumentCaptor.capture(), eq(bufferTimeoutMillis)); | ||
Record<Event> capturedRecord = argumentCaptor.getValue(); | ||
assertEquals( | ||
"DOCUMENT", | ||
capturedRecord.getData().getMetadata().getEventType(), | ||
"Event type should be 'DOCUMENT'" | ||
); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please use the longer copyright header.
https://github.com/opensearch-project/data-prepper/blob/90575b1de56f82f44d1af36f31ff4b077a627bd7/CONTRIBUTING.md#license-headers
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
added to every file