Refactor hoodie-hive
This commit is contained in:
committed by
prazanna
parent
c192dd60b4
commit
db6150c5ef
@@ -112,6 +112,10 @@ public class HoodieAvroDataBlock implements HoodieLogBlock {
|
||||
dis.readFully(compressedSchema, 0, schemaLength);
|
||||
Schema writerSchema = new Schema.Parser().parse(HoodieAvroUtils.decompress(compressedSchema));
|
||||
|
||||
if(readerSchema == null) {
|
||||
readerSchema = writerSchema;
|
||||
}
|
||||
|
||||
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
// 2. Get the total records
|
||||
int totalRecords = dis.readInt();
|
||||
|
||||
@@ -18,6 +18,14 @@ package com.uber.hoodie.common.util;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.net.URI;
|
||||
import java.nio.file.FileSystem;
|
||||
import java.nio.file.FileSystemNotFoundException;
|
||||
import java.nio.file.FileSystems;
|
||||
import java.nio.file.Path;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -29,7 +37,6 @@ import java.net.URISyntaxException;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
@@ -39,11 +46,6 @@ public class SchemaTestUtil {
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test.avro"));
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser()
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getSimpleSchema(), from, limit);
|
||||
@@ -53,11 +55,19 @@ public class SchemaTestUtil {
|
||||
int limit) throws IOException, URISyntaxException {
|
||||
GenericDatumReader<IndexedRecord> reader =
|
||||
new GenericDatumReader<>(writerSchema, readerSchema);
|
||||
try (Stream<String> stream = Files
|
||||
.lines(Paths.get(SchemaTestUtil.class.getResource("/sample.data").toURI()))) {
|
||||
// Required to register the necessary JAR:// file system
|
||||
URI resource = SchemaTestUtil.class.getClass().getResource("/sample.data").toURI();
|
||||
Path dataPath;
|
||||
if(resource.toString().contains("!")) {
|
||||
dataPath = uriToPath(resource);
|
||||
} else {
|
||||
dataPath = Paths.get(SchemaTestUtil.class.getClass().getResource("/sample.data").toURI());
|
||||
}
|
||||
|
||||
try (Stream<String> stream = Files.lines(dataPath)) {
|
||||
return stream.skip(from).limit(limit).map(s -> {
|
||||
try {
|
||||
return reader.read(null, DecoderFactory.get().jsonDecoder(readerSchema, s));
|
||||
return reader.read(null, DecoderFactory.get().jsonDecoder(writerSchema, s));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not read data from simple_data.json", e);
|
||||
}
|
||||
@@ -67,6 +77,18 @@ public class SchemaTestUtil {
|
||||
}
|
||||
}
|
||||
|
||||
static Path uriToPath(URI uri) throws IOException {
|
||||
final Map<String, String> env = new HashMap<>();
|
||||
final String[] array = uri.toString().split("!");
|
||||
FileSystem fs;
|
||||
try {
|
||||
fs = FileSystems.getFileSystem(URI.create(array[0]));
|
||||
} catch (FileSystemNotFoundException e) {
|
||||
fs = FileSystems.newFileSystem(URI.create(array[0]), env);
|
||||
}
|
||||
return fs.getPath(array[1]);
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateHoodieTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
List<IndexedRecord> records = generateTestRecords(from, limit);
|
||||
@@ -81,4 +103,14 @@ public class SchemaTestUtil {
|
||||
Collectors.toList());
|
||||
|
||||
}
|
||||
|
||||
public static Schema getEvolvedSchema() throws IOException {
|
||||
return new Schema.Parser()
|
||||
.parse(SchemaTestUtil.class.getResourceAsStream("/simple-test-evolved.avro"));
|
||||
}
|
||||
|
||||
public static List<IndexedRecord> generateEvolvedTestRecords(int from, int limit)
|
||||
throws IOException, URISyntaxException {
|
||||
return toRecords(getSimpleSchema(), getEvolvedSchema(), from, limit);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -7,6 +7,7 @@
|
||||
{"name": "field2", "type": ["null", "string"], "default": null},
|
||||
{"name": "name", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_number", "type": ["null", "long"], "default": null},
|
||||
{"name": "favorite_color", "type": ["null", "string"], "default": null}
|
||||
{"name": "favorite_color", "type": ["null", "string"], "default": null},
|
||||
{"name": "favorite_movie", "type": ["null", "string"], "default": null}
|
||||
]
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,7 +4,7 @@
|
||||
"name": "User",
|
||||
"fields": [
|
||||
{"name": "name", "type": "string"},
|
||||
{"name": "favorite_number", "type": "long"},
|
||||
{"name": "favorite_number", "type": "int"},
|
||||
{"name": "favorite_color", "type": "string"}
|
||||
]
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user