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

Remove EOFException catch block from the Avro decoders #15018

Merged
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.ParseException;

import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Map;
Expand Down Expand Up @@ -87,17 +87,8 @@ public GenericRecord parse(ByteBuffer bytes)
try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) {
return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null));
}
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
abhishekrb19 marked this conversation as resolved.
Show resolved Hide resolved
throw new ParseException(
null,
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (Exception e) {
throw new ParseException(null, e, "Fail to decode avro message!");
catch (IOException e) {
throw new ParseException(null, e, "Failed to decode Avro message");
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.common.parsers.ParseException;

import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
import java.util.Map;
Expand Down Expand Up @@ -119,17 +119,8 @@ public GenericRecord parse(ByteBuffer bytes)
try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) {
return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null));
}
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException(
null,
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (Exception e) {
throw new ParseException(null, e, "Fail to decode avro message with schemaId [%s].", schemaId);
catch (IOException ioe) {
throw new ParseException(null, ioe, "Failed to decode Avro message with schema id[%s]", schemaId);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,6 @@
import org.schemarepo.api.TypedSchemaRepository;
import org.schemarepo.api.converter.AvroSchemaConverter;

import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Collections;
Expand Down Expand Up @@ -83,17 +82,8 @@ public GenericRecord parse(ByteBuffer bytes)
try (ByteBufferInputStream inputStream = new ByteBufferInputStream(Collections.singletonList(bytes))) {
return reader.read(null, DecoderFactory.get().binaryDecoder(inputStream, null));
}
catch (EOFException eof) {
// waiting for avro v1.9.0 (#AVRO-813)
throw new ParseException(
null,
eof,
"Avro's unnecessary EOFException, detail: [%s]",
"https://issues.apache.org/jira/browse/AVRO-813"
);
}
catch (IOException e) {
throw new ParseException(null, e, "Fail to decode avro message!");
throw new ParseException(null, e, "Failed to decode Avro message");
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.apache.druid.data.input.AvroStreamInputRowParserTest;
import org.apache.druid.data.input.SomeAvroDatum;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.junit.Assert;
import org.junit.Test;

Expand Down Expand Up @@ -106,4 +107,77 @@ public void testParse() throws Exception
).parse(ByteBuffer.wrap(out.toByteArray()));
Assert.assertEquals(someAvroDatum.get("id"), actual.get("id"));
}

@Test
public void testParseWithInvalidVersion() throws Exception
{
GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum();
Schema schema = SomeAvroDatum.getClassSchema();

ByteArrayOutputStream out = new ByteArrayOutputStream();

DatumWriter<GenericRecord> writer = new SpecificDatumWriter<>(schema);
writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null));

ParseException parseException = Assert.assertThrows(
ParseException.class,
() -> new InlineSchemasAvroBytesDecoder(
ImmutableMap.of(
10,
schema
)
).parse(ByteBuffer.wrap(out.toByteArray()))
);
Assert.assertTrue(parseException.getMessage().contains("found record of arbitrary version"));
}

@Test
public void testParseWithInvalidSchemaId() throws Exception
{
GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum();
Schema schema = SomeAvroDatum.getClassSchema();

ByteArrayOutputStream out = new ByteArrayOutputStream();
out.write(new byte[]{1});

DatumWriter<GenericRecord> writer = new SpecificDatumWriter<>(schema);
writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null));

ParseException parseException = Assert.assertThrows(
ParseException.class,
() -> new InlineSchemasAvroBytesDecoder(
ImmutableMap.of(
10,
schema
)
).parse(ByteBuffer.wrap(out.toByteArray()))
);
Assert.assertTrue(parseException.getMessage().contains("Failed to find schema for id"));
}

@Test
public void testParseWithBadData() throws Exception
{
GenericRecord someAvroDatum = AvroStreamInputRowParserTest.buildSomeAvroDatum();
Schema schema = SomeAvroDatum.getClassSchema();

ByteArrayOutputStream out = new ByteArrayOutputStream();
out.write(new byte[]{1});
out.write(ByteBuffer.allocate(4).putInt(10).array());
out.write(ByteBuffer.allocate(24).putInt(777).array()); // add some junk

DatumWriter<GenericRecord> writer = new SpecificDatumWriter<>(schema);
writer.write(someAvroDatum, EncoderFactory.get().directBinaryEncoder(out, null));

ParseException parseException = Assert.assertThrows(
ParseException.class,
() -> new InlineSchemasAvroBytesDecoder(
ImmutableMap.of(
10,
schema
)
).parse(ByteBuffer.wrap(out.toByteArray()))
);
Assert.assertTrue(parseException.getMessage().contains("Failed to decode Avro message with schema id[10]"));
}
}
Loading