1
0

Adding a tool to read/inspect a HoodieLogFile

This commit is contained in:
Nishith Agarwal
2018-01-21 21:09:52 -08:00
committed by vinoth chandar
parent ba7c258c61
commit 9dff8c2326
5 changed files with 275 additions and 26 deletions

View File

@@ -395,21 +395,13 @@ public class HoodieHiveClient {
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLogFile(Optional<HoodieInstant> lastCompactionCommitOpt,
Path path) throws IOException {
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
lastBlock = (HoodieAvroDataBlock) block;
}
}
if (lastBlock != null) {
lastBlock.getRecords();
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
}
MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path);
// Fall back to read the schema from last compaction
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
return readSchemaFromLastCompaction(lastCompactionCommitOpt);
if (messageType == null) {
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
return readSchemaFromLastCompaction(lastCompactionCommitOpt);
}
return messageType;
}
/**

View File

@@ -18,13 +18,22 @@ package com.uber.hoodie.hive.util;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.uber.hoodie.common.model.HoodieLogFile;
import com.uber.hoodie.common.table.log.HoodieLogFormat;
import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
import com.uber.hoodie.common.table.timeline.HoodieInstant;
import com.uber.hoodie.hive.HiveSyncConfig;
import com.uber.hoodie.hive.HoodieHiveSyncException;
import com.uber.hoodie.hive.SchemaDifference;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.Set;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.schema.DecimalMetadata;
@@ -416,4 +425,24 @@ public class SchemaUtil {
// TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that
return "String";
}
/**
* Read the schema from the log file on path
*/
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
public static MessageType readSchemaFromLogFile(FileSystem fs,
Path path) throws IOException {
Reader reader = HoodieLogFormat.newReader(fs, new HoodieLogFile(path), null);
HoodieAvroDataBlock lastBlock = null;
while (reader.hasNext()) {
HoodieLogBlock block = reader.next();
if (block instanceof HoodieAvroDataBlock) {
lastBlock = (HoodieAvroDataBlock) block;
}
}
if (lastBlock != null) {
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
}
return null;
}
}