Skip to content
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

HDDS-10777. S3 gateway error when parsing xml in concurrent execution #6609

Merged
merged 3 commits into from
May 8, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -44,14 +44,13 @@ public class CompleteMultipartUploadRequestUnmarshaller
implements MessageBodyReader<CompleteMultipartUploadRequest> {

private final JAXBContext context;
private final XMLReader xmlReader;
private final SAXParserFactory saxParserFactory;

public CompleteMultipartUploadRequestUnmarshaller() {
try {
context = JAXBContext.newInstance(CompleteMultipartUploadRequest.class);
SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
xmlReader = saxParserFactory.newSAXParser().getXMLReader();
} catch (Exception ex) {
throw new AssertionError("Can not instantiate " +
"CompleteMultipartUploadRequest parser", ex);
Expand All @@ -70,6 +69,7 @@ public CompleteMultipartUploadRequest readFrom(
MultivaluedMap<String, String> multivaluedMap,
InputStream inputStream) throws IOException, WebApplicationException {
try {
XMLReader xmlReader = saxParserFactory.newSAXParser().getXMLReader();
UnmarshallerHandler unmarshallerHandler =
context.createUnmarshaller().getUnmarshallerHandler();
XmlNamespaceFilter filter =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,14 +43,13 @@ public class MultiDeleteRequestUnmarshaller
implements MessageBodyReader<MultiDeleteRequest> {

private final JAXBContext context;
private final XMLReader xmlReader;
private final SAXParserFactory saxParserFactory;

public MultiDeleteRequestUnmarshaller() {
try {
context = JAXBContext.newInstance(MultiDeleteRequest.class);
SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
xmlReader = saxParserFactory.newSAXParser().getXMLReader();
} catch (Exception ex) {
throw new AssertionError("Can't instantiate MultiDeleteRequest parser",
ex);
Expand All @@ -68,6 +67,7 @@ public MultiDeleteRequest readFrom(Class<MultiDeleteRequest> type,
Type genericType, Annotation[] annotations, MediaType mediaType,
MultivaluedMap<String, String> httpHeaders, InputStream entityStream) {
try {
XMLReader xmlReader = saxParserFactory.newSAXParser().getXMLReader();
UnmarshallerHandler unmarshallerHandler =
context.createUnmarshaller().getUnmarshallerHandler();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,14 +44,13 @@ public class PutBucketAclRequestUnmarshaller
implements MessageBodyReader<S3BucketAcl> {

private final JAXBContext context;
private final XMLReader xmlReader;
private final SAXParserFactory saxParserFactory;

public PutBucketAclRequestUnmarshaller() {
try {
context = JAXBContext.newInstance(S3BucketAcl.class);
SAXParserFactory saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory = SAXParserFactory.newInstance();
saxParserFactory.setFeature(XMLConstants.FEATURE_SECURE_PROCESSING, true);
xmlReader = saxParserFactory.newSAXParser().getXMLReader();
} catch (Exception ex) {
throw new AssertionError("Can not instantiate " +
"PutBucketAclRequest parser", ex);
Expand All @@ -70,6 +69,7 @@ public S3BucketAcl readFrom(
MultivaluedMap<String, String> multivaluedMap,
InputStream inputStream) throws IOException, WebApplicationException {
try {
XMLReader xmlReader = saxParserFactory.newSAXParser().getXMLReader();
UnmarshallerHandler unmarshallerHandler =
context.createUnmarshaller().getUnmarshallerHandler();
XmlNamespaceFilter filter =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,16 @@

import java.io.ByteArrayInputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;

import org.junit.jupiter.api.Test;

import static java.nio.charset.StandardCharsets.UTF_8;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;

/**
* Class tests Unmarshall logic of {@link CompleteMultipartUploadRequest}.
Expand Down Expand Up @@ -75,6 +79,7 @@ public void fromStreamWithoutNamespace() throws IOException {
}

private void checkContent(CompleteMultipartUploadRequest request) {
assertNotNull(request);
assertEquals(2, request.getPartList().size());

List<CompleteMultipartUploadRequest.Part> parts =
Expand All @@ -89,4 +94,43 @@ private CompleteMultipartUploadRequest unmarshall(
return new CompleteMultipartUploadRequestUnmarshaller()
.readFrom(null, null, null, null, null, inputBody);
}

@Test
public void concurrentParse() {
CompleteMultipartUploadRequestUnmarshaller unmarshaller =
new CompleteMultipartUploadRequestUnmarshaller();
byte[] bytes = ("<CompleteMultipartUpload>" + "<Part><ETag>" + part1 +
"</ETag><PartNumber>1</PartNumber" + "></Part><Part><ETag>" +
part2 + "</ETag><PartNumber>2" +
"</PartNumber></Part></CompleteMultipartUpload>").getBytes(
UTF_8);

List<CompletableFuture<CompleteMultipartUploadRequest>> futures =
new ArrayList<>();
for (int i = 0; i < 40; i++) {
futures.add(CompletableFuture.supplyAsync(() -> {
try {
//GIVEN
ByteArrayInputStream inputBody = new ByteArrayInputStream(bytes);
//WHEN
return unmarshall(unmarshaller, inputBody);
} catch (IOException e) {
return null;
}
}));
}

for (CompletableFuture<CompleteMultipartUploadRequest> future : futures) {
CompleteMultipartUploadRequest request = future.join();
//THEN
checkContent(request);
}
}

private CompleteMultipartUploadRequest unmarshall(
CompleteMultipartUploadRequestUnmarshaller unmarshaller,
ByteArrayInputStream inputBody) throws IOException {
return unmarshaller
.readFrom(null, null, null, null, null, inputBody);
}
}