1
0

- Ugrading to Hive 2.x

- Eliminating in-memory deltaRecordsMap
- Use writerSchema to generate generic record needed by custom payloads
- changes to make tests work with hive 2.x
This commit is contained in:
Nishith Agarwal
2019-05-10 13:09:09 -07:00
committed by vinoth chandar
parent cd7623e216
commit 129e433641
22 changed files with 554 additions and 191 deletions

View File

@@ -310,8 +310,6 @@ public abstract class AbstractHoodieLogRecordScanner {
processAvroDataBlock((HoodieAvroDataBlock) lastBlock);
break;
case DELETE_BLOCK:
// TODO : If delete is the only block written and/or records are present in parquet file
// TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
break;
case CORRUPT_BLOCK:

View File

@@ -331,6 +331,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
/**
* hasPrev is not idempotent
*/
@Override
public boolean hasPrev() {
try {
if (!this.reverseReader) {
@@ -352,6 +353,7 @@ class HoodieLogFileReader implements HoodieLogFormat.Reader {
* iterate reverse (prev) or forward (next). Doing both in the same instance is not supported
* WARNING : Every call to prev() should be preceded with hasPrev()
*/
@Override
public HoodieLogBlock prev() throws IOException {
if (!this.reverseReader) {

View File

@@ -81,6 +81,19 @@ public interface HoodieLogFormat {
* @return the path to this {@link HoodieLogFormat}
*/
HoodieLogFile getLogFile();
/**
* Read log file in reverse order and check if prev block is present
* @return
*/
public boolean hasPrev();
/**
* Read log file in reverse order and return prev block if present
* @return
* @throws IOException
*/
public HoodieLogBlock prev() throws IOException;
}
@@ -246,6 +259,13 @@ public interface HoodieLogFormat {
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE, false, false);
}
static HoodieLogFormat.Reader newReader(FileSystem fs, HoodieLogFile logFile, Schema readerSchema, boolean
readBlockLazily, boolean reverseReader)
throws IOException {
return new HoodieLogFileReader(fs, logFile, readerSchema, HoodieLogFileReader.DEFAULT_BUFFER_SIZE,
readBlockLazily, reverseReader);
}
/**
* A set of feature flags associated with a log format. Versions are changed when the log format
* changes. TODO(na) - Implement policies around major/minor versions

View File

@@ -119,4 +119,14 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
public void remove() {
}
@Override
public boolean hasPrev() {
return this.currentReader.hasPrev();
}
@Override
public HoodieLogBlock prev() throws IOException {
return this.currentReader.prev();
}
}

View File

@@ -44,6 +44,7 @@ import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DecoderFactory;
import org.apache.avro.io.EncoderFactory;
import org.codehaus.jackson.JsonNode;
import org.codehaus.jackson.node.NullNode;
/**
* Helper class to do common stuff across Avro.
@@ -156,16 +157,16 @@ public class HoodieAvroUtils {
* Add null fields to passed in schema. Caller is responsible for ensuring there is no duplicates.
* As different query engines have varying constraints regarding treating the case-sensitivity of fields, its best
* to let caller determine that.
*
* @param schema Passed in schema
* @param newFieldNames Null Field names to be added
* @return
*/
public static Schema appendNullSchemaFields(Schema schema, List<String> newFieldNames) {
List<Field> newFields = schema.getFields().stream().map(field -> {
return new Schema.Field(field.name(), field.schema(), field.doc(), field.defaultValue());
}).collect(Collectors.toList());
for (String newField : newFieldNames) {
newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", null));
newFields.add(new Schema.Field(newField, METADATA_FIELD_SCHEMA, "", NullNode.getInstance()));
}
Schema newSchema = Schema.createRecord(schema.getName(), schema.getDoc(), schema.getNamespace(), schema.isError());
newSchema.setFields(newFields);
@@ -184,11 +185,24 @@ public class HoodieAvroUtils {
/**
* Given a avro record with a given schema, rewrites it into the new schema
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the old
* schema
*/
public static GenericRecord rewriteRecord(GenericRecord record, Schema newSchema) {
return rewrite(record, record.getSchema(), newSchema);
}
/**
* Given a avro record with a given schema, rewrites it into the new schema while setting fields only from the new
* schema
*/
public static GenericRecord rewriteRecordWithOnlyNewSchemaFields(GenericRecord record, Schema newSchema) {
return rewrite(record, newSchema, newSchema);
}
private static GenericRecord rewrite(GenericRecord record, Schema schemaWithFields, Schema newSchema) {
GenericRecord newRecord = new GenericData.Record(newSchema);
for (Schema.Field f : record.getSchema().getFields()) {
for (Schema.Field f : schemaWithFields.getFields()) {
newRecord.put(f.name(), record.get(f.name()));
}
if (!GenericData.get().validate(newSchema, newRecord)) {

View File

@@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.uber.hoodie.common.util;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.HoodieTableMetaClient;
import com.uber.hoodie.common.table.HoodieTimeline;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
import java.io.IOException;
import java.util.List;
import java.util.stream.Collectors;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
/**
* Utils class for performing various log file reading operations
*/
public class LogReaderUtils {
private static Schema readSchemaFromLogFileInReverse(FileSystem fs, HoodieActiveTimeline activeTimeline, Path path)
throws IOException {
HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null, true, true);
Schema writerSchema = null;
HoodieTimeline completedTimeline = activeTimeline.getCommitsTimeline().filterCompletedInstants();
while (reader.hasPrev()) {
HoodieLogBlock block = reader.prev();
if (block instanceof HoodieAvroDataBlock && block != null) {
HoodieAvroDataBlock lastBlock = (HoodieAvroDataBlock) block;
if (completedTimeline.containsOrBeforeTimelineStarts(lastBlock.getLogBlockHeader().get(HeaderMetadataType
.INSTANT_TIME))) {
writerSchema = Schema.parse(lastBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
break;
}
}
}
reader.close();
return writerSchema;
}
public static Schema readLatestSchemaFromLogFiles(String basePath, List<String> deltaFilePaths, JobConf jobConf)
throws IOException {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jobConf, basePath);
List<String> deltaPaths = deltaFilePaths.stream().map(s -> new HoodieLogFile(new Path(s)))
.sorted(HoodieLogFile.getReverseLogFileComparator()).map(s -> s.getPath().toString())
.collect(Collectors.toList());
if (deltaPaths.size() > 0) {
for (String logPath : deltaPaths) {
FileSystem fs = FSUtils.getFs(logPath, jobConf);
Schema schemaFromLogFile =
readSchemaFromLogFileInReverse(fs, metaClient.getActiveTimeline(), new Path(logPath));
if (schemaFromLogFile != null) {
return schemaFromLogFile;
}
}
}
return null;
}
}