Initial impl of HoodieRealtimeInputFormat
- Works end-end for flat schemas - Schema evolution & hardening remains - HoodieClientExample can now write mor tables as well
This commit is contained in:
@@ -131,7 +131,11 @@ public class HoodiePartitionMetadata {
|
||||
}
|
||||
|
||||
// methods related to partition meta data
|
||||
public static boolean hasPartitionMetadata(FileSystem fs, Path partitionPath) throws IOException {
|
||||
return fs.exists(new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
|
||||
public static boolean hasPartitionMetadata(FileSystem fs, Path partitionPath) {
|
||||
try {
|
||||
return fs.exists(new Path(partitionPath, HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException("Error checking Hoodie partition metadata for " + partitionPath, ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -98,7 +98,7 @@ public interface TableFileSystemView {
|
||||
|
||||
/**
|
||||
* Group data files with corresponding delta files
|
||||
* @param fs
|
||||
*
|
||||
* @param partitionPath
|
||||
* @return
|
||||
* @throws IOException
|
||||
|
||||
@@ -207,7 +207,7 @@ public class HoodieTableFileSystemView implements TableFileSystemView, Serializa
|
||||
l -> l.stream().sorted(HoodieLogFile.getLogVersionComparator())
|
||||
.collect(toList())));
|
||||
|
||||
// Filter the delta files by the commit time of the latest base fine and collect as a list
|
||||
// Filter the delta files by the commit time of the latest base file and collect as a list
|
||||
Optional<HoodieInstant> lastTimestamp = metaClient.getActiveTimeline().lastInstant();
|
||||
return lastTimestamp.map(hoodieInstant -> getLatestVersionInPartition(partitionPath,
|
||||
hoodieInstant.getTimestamp()).map(
|
||||
|
||||
@@ -135,6 +135,12 @@ public class FSUtils {
|
||||
return datePartitions;
|
||||
}
|
||||
|
||||
public static String getRelativePartitionPath(Path basePath, Path partitionPath) {
|
||||
String partitionFullPath = partitionPath.toString();
|
||||
int partitionStartIndex = partitionFullPath.lastIndexOf(basePath.getName());
|
||||
return partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all the partition paths, that are present in this table, denoted by presence of {@link
|
||||
* com.uber.hoodie.common.model.HoodiePartitionMetadata#HOODIE_PARTITION_METAFILE}
|
||||
@@ -147,9 +153,7 @@ public class FSUtils {
|
||||
while (allFiles.hasNext()) {
|
||||
Path filePath = allFiles.next().getPath();
|
||||
if (filePath.getName().equals(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE)) {
|
||||
String partitionFullPath = filePath.getParent().toString();
|
||||
int partitionStartIndex = partitionFullPath.lastIndexOf(basePath.getName()) ;
|
||||
partitions.add(partitionFullPath.substring(partitionStartIndex + basePath.getName().length() + 1));
|
||||
partitions.add(getRelativePartitionPath(basePath, filePath.getParent()));
|
||||
}
|
||||
}
|
||||
return partitions;
|
||||
|
||||
@@ -31,6 +31,7 @@ import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetFileReader;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
|
||||
import org.apache.parquet.schema.MessageType;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.ArrayList;
|
||||
@@ -80,17 +81,48 @@ public class ParquetUtils {
|
||||
return rowKeys;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
*
|
||||
* Read the metadata from a parquet file
|
||||
*
|
||||
* @param parquetFilePath
|
||||
* @return
|
||||
*/
|
||||
public static ParquetMetadata readMetadata(Path parquetFilePath) {
|
||||
return readMetadata(new Configuration(), parquetFilePath);
|
||||
}
|
||||
|
||||
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
|
||||
ParquetMetadata footer;
|
||||
try {
|
||||
// TODO(vc): Should we use the parallel reading version here?
|
||||
footer = ParquetFileReader.readFooter(conf, parquetFilePath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to read footer for parquet " + parquetFilePath,
|
||||
e);
|
||||
}
|
||||
return footer;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Get the schema of the given parquet file.
|
||||
*
|
||||
* @param parquetFilePath
|
||||
* @return
|
||||
*/
|
||||
public static MessageType readSchema(Path parquetFilePath) {
|
||||
return readMetadata(parquetFilePath).getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
|
||||
|
||||
/**
|
||||
* Read out the bloom filter from the parquet file meta data.
|
||||
*/
|
||||
public static BloomFilter readBloomFilterFromParquetMetadata(Path parquetFilePath) {
|
||||
ParquetMetadata footer;
|
||||
try {
|
||||
footer = ParquetFileReader.readFooter(new Configuration(), parquetFilePath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIndexException("Failed to read footer for parquet " + parquetFilePath,
|
||||
e);
|
||||
}
|
||||
ParquetMetadata footer = readMetadata(parquetFilePath);
|
||||
Map<String, String> metadata = footer.getFileMetaData().getKeyValueMetaData();
|
||||
if (metadata.containsKey(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY)) {
|
||||
return new BloomFilter(metadata.get(HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY));
|
||||
|
||||
Reference in New Issue
Block a user