1
0

[HUDI-1533] Make SerializableSchema work for large schemas and add ability to sortBy numeric values (#2453)

This commit is contained in:
satishkotha
2021-01-17 12:36:55 -08:00
committed by GitHub
parent a43e191d6c
commit 3d1d5d00b0
3 changed files with 40 additions and 9 deletions

View File

@@ -62,12 +62,17 @@ public class SerializableSchema implements Serializable {
// create a public write method for unit test
public void writeObjectTo(ObjectOutputStream out) throws IOException {
out.writeUTF(schema.toString());
// Note: writeUTF cannot support string length > 64K. So use writeObject which has small overhead (relatively).
out.writeObject(schema.toString());
}
// create a public read method for unit test
public void readObjectFrom(ObjectInputStream in) throws IOException {
schema = new Schema.Parser().parse(in.readUTF());
try {
schema = new Schema.Parser().parse(in.readObject().toString());
} catch (ClassNotFoundException e) {
throw new IOException("unable to parse schema", e);
}
}
@Override

View File

@@ -49,6 +49,11 @@ public class TestSerializableSchema {
verifySchema(HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS);
}
@Test
public void testLargeSchema() throws IOException {
verifySchema(new Schema.Parser().parse(generateLargeSchema()));
}
private void verifySchema(Schema schema) throws IOException {
SerializableSchema serializableSchema = new SerializableSchema(schema);
assertEquals(schema, serializableSchema.get());
@@ -65,4 +70,20 @@ public class TestSerializableSchema {
newSchema.readObjectFrom(new ObjectInputStream(new ByteArrayInputStream(bytesWritten)));
assertEquals(schema, newSchema.get());
}
// generate large schemas (>64K which is limitation of ObjectOutputStream#writeUTF) to validate it can be serialized
private String generateLargeSchema() {
StringBuilder schema = new StringBuilder();
schema.append(HoodieTestDataGenerator.TRIP_SCHEMA_PREFIX);
int fieldNum = 1;
while (schema.length() < 80 * 1024) {
String fieldName = "field" + fieldNum;
schema.append("{\"name\": \"" + fieldName + "\",\"type\": {\"type\":\"record\", \"name\":\"" + fieldName + "\",\"fields\": ["
+ "{\"name\": \"amount\",\"type\": \"double\"},{\"name\": \"currency\", \"type\": \"string\"}]}},");
fieldNum++;
}
schema.append(HoodieTestDataGenerator.TRIP_SCHEMA_SUFFIX);
return schema.toString();
}
}