[HUDI-1533] Make SerializableSchema work for large schemas and add ability to sortBy numeric values (#2453)
This commit is contained in:
@@ -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
|
||||
|
||||
Reference in New Issue
Block a user