1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -19,7 +19,6 @@
package com.uber.hoodie.hive;
import com.beust.jcommander.Parameter;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;

View File

@@ -26,6 +26,10 @@ import com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat;
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent;
import com.uber.hoodie.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import com.uber.hoodie.hive.util.SchemaUtil;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Partition;
@@ -35,20 +39,14 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.schema.MessageType;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
/**
* Tool to sync a hoodie HDFS dataset with a hive metastore table.
* Either use it as a api HiveSyncTool.syncHoodieTable(HiveSyncConfig)
* or as a command line java -cp hoodie-hive.jar HiveSyncTool [args]
* Tool to sync a hoodie HDFS dataset with a hive metastore table. Either use it as a api
* HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive.jar
* HiveSyncTool [args]
*
* This utility will get the schema from the latest commit and will sync hive table schema
* Also this will sync the partitions incrementally
* (all the partitions modified since the last commit)
* This utility will get the schema from the latest commit and will sync hive table schema Also this
* will sync the partitions incrementally (all the partitions modified since the last commit)
*/
@SuppressWarnings("WeakerAccess")
public class HiveSyncTool {
@@ -64,7 +62,7 @@ public class HiveSyncTool {
}
public void syncHoodieTable() {
switch(hoodieHiveClient.getTableType()) {
switch (hoodieHiveClient.getTableType()) {
case COPY_ON_WRITE:
syncHoodieTable(false);
break;
@@ -125,15 +123,15 @@ public class HiveSyncTool {
// Check and sync schema
if (!tableExists) {
LOG.info("Table " + cfg.tableName + " is not found. Creating it");
if(!isRealTime) {
if (!isRealTime) {
// TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default for now)
hoodieHiveClient.createTable(schema, HoodieInputFormat.class.getName(),
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
} else {
// Custom serde will not work with ALTER TABLE REPLACE COLUMNS
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java#L3488
hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(),
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
// Custom serde will not work with ALTER TABLE REPLACE COLUMNS
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java#L3488
hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(),
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
}
} else {
// Check if the dataset schema has evolved

View File

@@ -198,8 +198,8 @@ public class HoodieHiveClient {
}
/**
* Iterate over the storage partitions and find if there are any new partitions that need
* to be added or updated. Generate a list of PartitionEvent based on the changes required.
* Iterate over the storage partitions and find if there are any new partitions that need to be
* added or updated. Generate a list of PartitionEvent based on the changes required.
*/
List<PartitionEvent> getPartitionEvents(List<Partition> tablePartitions,
List<String> partitionStoragePartitions) {
@@ -297,9 +297,9 @@ public class HoodieHiveClient {
}
/**
* Gets the schema for a hoodie dataset.
* Depending on the type of table, read from any file written in the latest commit.
* We will assume that the schema has not changed within a single atomic write.
* Gets the schema for a hoodie dataset. Depending on the type of table, read from any file
* written in the latest commit. We will assume that the schema has not changed within a single
* atomic write.
*
* @return Parquet schema for this dataset
*/
@@ -313,7 +313,8 @@ public class HoodieHiveClient {
.orElseThrow(() -> new InvalidDatasetException(syncConfig.basePath));
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCommit).get());
String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
.stream().findAny()
.orElseThrow(() -> new IllegalArgumentException(
"Could not find any data file written for commit " + lastCommit
+ ", could not get schema for dataset " + metaClient.getBasePath()));
@@ -330,7 +331,8 @@ public class HoodieHiveClient {
lastDeltaCommitAfterCompaction = metaClient.getActiveTimeline()
.getDeltaCommitTimeline()
.filterCompletedInstants()
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE).lastInstant();
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE)
.lastInstant();
}
LOG.info("Found the last delta commit after last compaction as "
+ lastDeltaCommitAfterCompaction);
@@ -340,8 +342,9 @@ public class HoodieHiveClient {
// read from the log file wrote
commitMetadata = HoodieCommitMetadata
.fromBytes(activeTimeline.getInstantDetails(lastDeltaCommit).get());
filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().filter(s -> s.contains(
HoodieLogFile.DELTA_EXTENSION)).findAny()
filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
.stream().filter(s -> s.contains(
HoodieLogFile.DELTA_EXTENSION)).findAny()
.orElseThrow(() -> new IllegalArgumentException(
"Could not find any data file written for commit " + lastDeltaCommit
+ ", could not get schema for dataset " + metaClient.getBasePath()));
@@ -361,10 +364,6 @@ public class HoodieHiveClient {
/**
* Read schema from a data file from the last compaction commit done.
*
* @param lastCompactionCommitOpt
* @return
* @throws IOException
*/
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLastCompaction(Optional<HoodieInstant> lastCompactionCommitOpt)
@@ -377,7 +376,8 @@ public class HoodieHiveClient {
// Read from the compacted file wrote
HoodieCompactionMetadata compactionMetadata = HoodieCompactionMetadata
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get());
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny()
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
.stream().findAny()
.orElseThrow(() -> new IllegalArgumentException(
"Could not find any data file written for compaction " + lastCompactionCommit
+ ", could not get schema for dataset " + metaClient.getBasePath()));
@@ -386,11 +386,6 @@ public class HoodieHiveClient {
/**
* Read the schema from the log file on path
*
* @param lastCompactionCommitOpt
* @param path
* @return
* @throws IOException
*/
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLogFile(Optional<HoodieInstant> lastCompactionCommitOpt,
@@ -422,7 +417,8 @@ public class HoodieHiveClient {
+ ". File does not exist.");
}
ParquetMetadata fileFooter =
ParquetFileReader.readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER);
ParquetFileReader
.readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER);
return fileFooter.getFileMetaData().getSchema();
}
@@ -530,7 +526,7 @@ public class HoodieHiveClient {
if (connection != null) {
connection.close();
}
if(client != null) {
if (client != null) {
client.close();
}
} catch (SQLException e) {

View File

@@ -18,23 +18,23 @@ package com.uber.hoodie.hive;
public class HoodieHiveSyncException extends RuntimeException {
public HoodieHiveSyncException() {
super();
}
public HoodieHiveSyncException() {
super();
}
public HoodieHiveSyncException(String message) {
super(message);
}
public HoodieHiveSyncException(String message) {
super(message);
}
public HoodieHiveSyncException(String message, Throwable t) {
super(message, t);
}
public HoodieHiveSyncException(String message, Throwable t) {
super(message, t);
}
public HoodieHiveSyncException(Throwable t) {
super(t);
}
public HoodieHiveSyncException(Throwable t) {
super(t);
}
protected static String format(String message, Object... args) {
return String.format(String.valueOf(message), (Object[]) args);
}
protected static String format(String message, Object... args) {
return String.format(String.valueOf(message), (Object[]) args);
}
}

View File

@@ -21,11 +21,13 @@ package com.uber.hoodie.hive;
import java.util.List;
/**
* HDFS Path contain hive partition values for the keys it is partitioned on.
* This mapping is not straight forward and requires a pluggable implementation to extract the partition value from HDFS path.
* HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not
* straight forward and requires a pluggable implementation to extract the partition value from HDFS
* path.
*
* e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd]
*/
public interface PartitionValueExtractor {
List<String> extractPartitionValuesInPath(String partitionPath);
}

View File

@@ -21,88 +21,92 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import parquet.schema.MessageType;
import java.util.List;
import java.util.Map;
import parquet.schema.MessageType;
/**
* Represents the schema difference between the storage schema and hive table schema
*/
public class SchemaDifference {
private final MessageType storageSchema;
private final Map<String, String> tableSchema;
private final List<String> deleteColumns;
private final Map<String, String> updateColumnTypes;
private final Map<String, String> addColumnTypes;
private SchemaDifference(MessageType storageSchema, Map<String, String> tableSchema,
List<String> deleteColumns, Map<String, String> updateColumnTypes,
Map<String, String> addColumnTypes) {
this.storageSchema = storageSchema;
this.tableSchema = tableSchema;
this.deleteColumns = ImmutableList.copyOf(deleteColumns);
this.updateColumnTypes = ImmutableMap.copyOf(updateColumnTypes);
this.addColumnTypes = ImmutableMap.copyOf(addColumnTypes);
}
public List<String> getDeleteColumns() {
return deleteColumns;
}
public Map<String, String> getUpdateColumnTypes() {
return updateColumnTypes;
}
public Map<String, String> getAddColumnTypes() {
return addColumnTypes;
}
@Override
public String toString() {
return Objects.toStringHelper(this).add("deleteColumns", deleteColumns)
.add("updateColumnTypes", updateColumnTypes).add("addColumnTypes", addColumnTypes)
.toString();
}
public static Builder newBuilder(MessageType storageSchema, Map<String, String> tableSchema) {
return new Builder(storageSchema, tableSchema);
}
public boolean isEmpty() {
return deleteColumns.isEmpty() && updateColumnTypes.isEmpty() && addColumnTypes.isEmpty();
}
public static class Builder {
private final MessageType storageSchema;
private final Map<String, String> tableSchema;
private final List<String> deleteColumns;
private final Map<String, String> updateColumnTypes;
private final Map<String, String> addColumnTypes;
private List<String> deleteColumns;
private Map<String, String> updateColumnTypes;
private Map<String, String> addColumnTypes;
private SchemaDifference(MessageType storageSchema, Map<String, String> tableSchema,
List<String> deleteColumns, Map<String, String> updateColumnTypes, Map<String, String> addColumnTypes) {
this.storageSchema = storageSchema;
this.tableSchema = tableSchema;
this.deleteColumns = ImmutableList.copyOf(deleteColumns);
this.updateColumnTypes = ImmutableMap.copyOf(updateColumnTypes);
this.addColumnTypes = ImmutableMap.copyOf(addColumnTypes);
public Builder(MessageType storageSchema, Map<String, String> tableSchema) {
this.storageSchema = storageSchema;
this.tableSchema = tableSchema;
deleteColumns = Lists.newArrayList();
updateColumnTypes = Maps.newHashMap();
addColumnTypes = Maps.newHashMap();
}
public List<String> getDeleteColumns() {
return deleteColumns;
public Builder deleteTableColumn(String column) {
deleteColumns.add(column);
return this;
}
public Map<String, String> getUpdateColumnTypes() {
return updateColumnTypes;
public Builder updateTableColumn(String column, String storageColumnType) {
updateColumnTypes.put(column, storageColumnType);
return this;
}
public Map<String, String> getAddColumnTypes() {
return addColumnTypes;
public Builder addTableColumn(String name, String type) {
addColumnTypes.put(name, type);
return this;
}
@Override public String toString() {
return Objects.toStringHelper(this).add("deleteColumns", deleteColumns)
.add("updateColumnTypes", updateColumnTypes).add("addColumnTypes", addColumnTypes)
.toString();
}
public static Builder newBuilder(MessageType storageSchema, Map<String, String> tableSchema) {
return new Builder(storageSchema, tableSchema);
}
public boolean isEmpty() {
return deleteColumns.isEmpty() && updateColumnTypes.isEmpty() && addColumnTypes.isEmpty();
}
public static class Builder {
private final MessageType storageSchema;
private final Map<String, String> tableSchema;
private List<String> deleteColumns;
private Map<String, String> updateColumnTypes;
private Map<String, String> addColumnTypes;
public Builder(MessageType storageSchema, Map<String, String> tableSchema) {
this.storageSchema = storageSchema;
this.tableSchema = tableSchema;
deleteColumns = Lists.newArrayList();
updateColumnTypes = Maps.newHashMap();
addColumnTypes = Maps.newHashMap();
}
public Builder deleteTableColumn(String column) {
deleteColumns.add(column);
return this;
}
public Builder updateTableColumn(String column, String storageColumnType) {
updateColumnTypes.put(column, storageColumnType);
return this;
}
public Builder addTableColumn(String name, String type) {
addColumnTypes.put(name, type);
return this;
}
public SchemaDifference build() {
return new SchemaDifference(storageSchema, tableSchema, deleteColumns, updateColumnTypes, addColumnTypes);
}
public SchemaDifference build() {
return new SchemaDifference(storageSchema, tableSchema, deleteColumns, updateColumnTypes,
addColumnTypes);
}
}
}

View File

@@ -23,9 +23,11 @@ import java.util.List;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
/**
* HDFS Path contain hive partition values for the keys it is partitioned on.
* This mapping is not straight forward and requires a pluggable implementation to extract the partition value from HDFS path.
* HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not
* straight forward and requires a pluggable implementation to extract the partition value from HDFS
* path.
*
* This implementation extracts datestr=yyyy-mm-dd from path of type /yyyy/mm/dd
*/

View File

@@ -17,32 +17,32 @@
package com.uber.hoodie.hive.util;
import com.google.common.collect.Maps;
import java.util.Iterator;
import java.util.Map;
public class ColumnNameXLator {
private static Map<String, String> xformMap = Maps.newHashMap();
public static String translateNestedColumn(String colName) {
Map.Entry entry;
for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext();
colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) {
entry = (Map.Entry) i$.next();
}
private static Map<String, String> xformMap = Maps.newHashMap();
return colName;
public static String translateNestedColumn(String colName) {
Map.Entry entry;
for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext();
colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) {
entry = (Map.Entry) i$.next();
}
public static String translateColumn(String colName) {
return colName;
}
return colName;
}
public static String translate(String colName, boolean nestedColumn) {
return !nestedColumn ? translateColumn(colName) : translateNestedColumn(colName);
}
public static String translateColumn(String colName) {
return colName;
}
static {
xformMap.put("\\$", "_dollar_");
}
public static String translate(String colName, boolean nestedColumn) {
return !nestedColumn ? translateColumn(colName) : translateNestedColumn(colName);
}
static {
xformMap.put("\\$", "_dollar_");
}
}

View File

@@ -21,6 +21,10 @@ import com.google.common.collect.Sets;
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.Set;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.schema.DecimalMetadata;
@@ -30,404 +34,386 @@ import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
import parquet.schema.Type;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
* Schema Utilities
*/
public class SchemaUtil {
private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
/**
* Get the schema difference between the storage schema and hive table schema
*
* @param storageSchema
* @param tableSchema
* @param partitionKeys
* @return
*/
public static SchemaDifference getSchemaDifference(MessageType storageSchema,
Map<String, String> tableSchema, List<String> partitionKeys) {
Map<String, String> newTableSchema;
try {
newTableSchema = convertParquetSchemaToHiveSchema(storageSchema);
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema",
e);
private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
/**
* Get the schema difference between the storage schema and hive table schema
*/
public static SchemaDifference getSchemaDifference(MessageType storageSchema,
Map<String, String> tableSchema, List<String> partitionKeys) {
Map<String, String> newTableSchema;
try {
newTableSchema = convertParquetSchemaToHiveSchema(storageSchema);
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema",
e);
}
LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema);
SchemaDifference.Builder schemaDiffBuilder =
SchemaDifference.newBuilder(storageSchema, tableSchema);
Set<String> tableColumns = Sets.newHashSet();
for (Map.Entry<String, String> field : tableSchema.entrySet()) {
String fieldName = field.getKey().toLowerCase();
String tickSurroundedFieldName = tickSurround(fieldName);
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !partitionKeys
.contains(fieldName)) {
schemaDiffBuilder.deleteTableColumn(fieldName);
} else {
// check type
String tableColumnType = field.getValue();
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName)) {
if (partitionKeys.contains(fieldName)) {
// Partition key does not have to be part of the storage schema
continue;
}
// We will log this and continue. Hive schema is a superset of all parquet schemas
LOG.warn("Ignoring table column " + fieldName
+ " as its not present in the parquet schema");
continue;
}
LOG.info("Getting schema difference for " + tableSchema + "\r\n\r\n" + newTableSchema);
SchemaDifference.Builder schemaDiffBuilder =
SchemaDifference.newBuilder(storageSchema, tableSchema);
Set<String> tableColumns = Sets.newHashSet();
tableColumnType = tableColumnType.replaceAll("\\s+", "");
for (Map.Entry<String, String> field : tableSchema.entrySet()) {
String fieldName = field.getKey().toLowerCase();
String tickSurroundedFieldName = tickSurround(fieldName);
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName) && !partitionKeys.contains(fieldName)) {
schemaDiffBuilder.deleteTableColumn(fieldName);
} else {
// check type
String tableColumnType = field.getValue();
if (!isFieldExistsInSchema(newTableSchema, tickSurroundedFieldName)) {
if (partitionKeys.contains(fieldName)) {
// Partition key does not have to be part of the storage schema
continue;
}
// We will log this and continue. Hive schema is a superset of all parquet schemas
LOG.warn("Ignoring table column " + fieldName
+ " as its not present in the parquet schema");
continue;
}
tableColumnType = tableColumnType.replaceAll("\\s+", "");
String expectedType = getExpectedType(newTableSchema, tickSurroundedFieldName);
expectedType = expectedType.replaceAll("\\s+", "");
expectedType = expectedType.replaceAll("`", "");
String expectedType = getExpectedType(newTableSchema, tickSurroundedFieldName);
expectedType = expectedType.replaceAll("\\s+", "");
expectedType = expectedType.replaceAll("`", "");
if (!tableColumnType.equalsIgnoreCase(expectedType)) {
// check for incremental datasets, the schema type change is allowed as per evolution rules
if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) {
throw new HoodieHiveSyncException(
"Could not convert field Type from " + tableColumnType + " to "
+ expectedType + " for field " + fieldName);
}
schemaDiffBuilder.updateTableColumn(fieldName,
getExpectedType(newTableSchema, tickSurroundedFieldName));
}
}
tableColumns.add(tickSurroundedFieldName);
if (!tableColumnType.equalsIgnoreCase(expectedType)) {
// check for incremental datasets, the schema type change is allowed as per evolution rules
if (!isSchemaTypeUpdateAllowed(tableColumnType, expectedType)) {
throw new HoodieHiveSyncException(
"Could not convert field Type from " + tableColumnType + " to "
+ expectedType + " for field " + fieldName);
}
schemaDiffBuilder.updateTableColumn(fieldName,
getExpectedType(newTableSchema, tickSurroundedFieldName));
}
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
if (!tableColumns.contains(entry.getKey().toLowerCase())) {
schemaDiffBuilder.addTableColumn(entry.getKey(), entry.getValue());
}
}
LOG.info("Difference between schemas: " + schemaDiffBuilder.build().toString());
return schemaDiffBuilder.build();
}
tableColumns.add(tickSurroundedFieldName);
}
private static String getExpectedType(Map<String, String> newTableSchema, String fieldName) {
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
if (entry.getKey().toLowerCase().equals(fieldName)) {
return entry.getValue();
}
}
return null;
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
if (!tableColumns.contains(entry.getKey().toLowerCase())) {
schemaDiffBuilder.addTableColumn(entry.getKey(), entry.getValue());
}
}
LOG.info("Difference between schemas: " + schemaDiffBuilder.build().toString());
private static boolean isFieldExistsInSchema(Map<String, String> newTableSchema,
String fieldName) {
for (String entry : newTableSchema.keySet()) {
if (entry.toLowerCase().equals(fieldName)) {
return true;
}
}
return false;
return schemaDiffBuilder.build();
}
private static String getExpectedType(Map<String, String> newTableSchema, String fieldName) {
for (Map.Entry<String, String> entry : newTableSchema.entrySet()) {
if (entry.getKey().toLowerCase().equals(fieldName)) {
return entry.getValue();
}
}
return null;
}
private static boolean isFieldExistsInSchema(Map<String, String> newTableSchema,
String fieldName) {
for (String entry : newTableSchema.keySet()) {
if (entry.toLowerCase().equals(fieldName)) {
return true;
}
}
return false;
}
/**
* Returns equivalent Hive table schema read from a parquet file
*
* @param messageType : Parquet Schema
* @return : Hive Table schema read from parquet file MAP[String,String]
* @throws IOException
*/
public static Map<String, String> convertParquetSchemaToHiveSchema(MessageType messageType)
throws IOException {
Map<String, String> schema = Maps.newLinkedHashMap();
List<Type> parquetFields = messageType.getFields();
for (Type parquetType : parquetFields) {
StringBuilder result = new StringBuilder();
String key = parquetType.getName();
if (parquetType.isRepetition(Type.Repetition.REPEATED)) {
result.append(createHiveArray(parquetType, ""));
} else {
result.append(convertField(parquetType));
/**
* Returns equivalent Hive table schema read from a parquet file
*
* @param messageType : Parquet Schema
* @return : Hive Table schema read from parquet file MAP[String,String]
*/
public static Map<String, String> convertParquetSchemaToHiveSchema(MessageType messageType)
throws IOException {
Map<String, String> schema = Maps.newLinkedHashMap();
List<Type> parquetFields = messageType.getFields();
for (Type parquetType : parquetFields) {
StringBuilder result = new StringBuilder();
String key = parquetType.getName();
if (parquetType.isRepetition(Type.Repetition.REPEATED)) {
result.append(createHiveArray(parquetType, ""));
} else {
result.append(convertField(parquetType));
}
schema.put(hiveCompatibleFieldName(key, false), result.toString());
}
return schema;
}
/**
* Convert one field data type of parquet schema into an equivalent Hive schema
*
* @param parquetType : Single paruet field
* @return : Equivalent sHive schema
*/
private static String convertField(final Type parquetType) {
StringBuilder field = new StringBuilder();
if (parquetType.isPrimitive()) {
final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName =
parquetType.asPrimitiveType().getPrimitiveTypeName();
final OriginalType originalType = parquetType.getOriginalType();
if (originalType == OriginalType.DECIMAL) {
final DecimalMetadata decimalMetadata =
parquetType.asPrimitiveType().getDecimalMetadata();
return field.append("DECIMAL(").append(decimalMetadata.getPrecision()).
append(" , ").append(decimalMetadata.getScale()).append(")").toString();
}
// TODO - fix the method naming here
return parquetPrimitiveTypeName
.convert(new PrimitiveType.PrimitiveTypeNameConverter<String, RuntimeException>() {
@Override
public String convertBOOLEAN(
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "boolean";
}
schema.put(hiveCompatibleFieldName(key, false), result.toString());
}
return schema;
}
/**
* Convert one field data type of parquet schema into an equivalent Hive
* schema
*
* @param parquetType : Single paruet field
* @return : Equivalent sHive schema
*/
private static String convertField(final Type parquetType) {
StringBuilder field = new StringBuilder();
if (parquetType.isPrimitive()) {
final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName =
parquetType.asPrimitiveType().getPrimitiveTypeName();
final OriginalType originalType = parquetType.getOriginalType();
if (originalType == OriginalType.DECIMAL) {
final DecimalMetadata decimalMetadata =
parquetType.asPrimitiveType().getDecimalMetadata();
return field.append("DECIMAL(").append(decimalMetadata.getPrecision()).
append(" , ").append(decimalMetadata.getScale()).append(")").toString();
@Override
public String convertINT32(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "int";
}
// TODO - fix the method naming here
return parquetPrimitiveTypeName
.convert(new PrimitiveType.PrimitiveTypeNameConverter<String, RuntimeException>() {
@Override
public String convertBOOLEAN(
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "boolean";
}
@Override
public String convertINT32(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "int";
}
@Override
public String convertINT64(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "bigint";
}
@Override
public String convertINT96(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "timestamp-millis";
}
@Override
public String convertFLOAT(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "float";
}
@Override
public String convertDOUBLE(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "double";
}
@Override
public String convertFIXED_LEN_BYTE_ARRAY(
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "binary";
}
@Override
public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
if (originalType == OriginalType.UTF8
|| originalType == OriginalType.ENUM) {
return "string";
} else {
return "binary";
}
}
});
} else {
GroupType parquetGroupType = parquetType.asGroupType();
OriginalType originalType = parquetGroupType.getOriginalType();
if (originalType != null) {
switch (originalType) {
case LIST:
if (parquetGroupType.getFieldCount() != 1) {
throw new UnsupportedOperationException(
"Invalid list type " + parquetGroupType);
}
Type elementType = parquetGroupType.getType(0);
if (!elementType.isRepetition(Type.Repetition.REPEATED)) {
throw new UnsupportedOperationException(
"Invalid list type " + parquetGroupType);
}
return createHiveArray(elementType, parquetGroupType.getName());
case MAP:
if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0)
.isPrimitive()) {
throw new UnsupportedOperationException(
"Invalid map type " + parquetGroupType);
}
GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType();
if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) ||
!mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) ||
mapKeyValType.getFieldCount() != 2) {
throw new UnsupportedOperationException(
"Invalid map type " + parquetGroupType);
}
Type keyType = mapKeyValType.getType(0);
if (!keyType.isPrimitive() ||
!keyType.asPrimitiveType().getPrimitiveTypeName()
.equals(PrimitiveType.PrimitiveTypeName.BINARY) ||
!keyType.getOriginalType().equals(OriginalType.UTF8)) {
throw new UnsupportedOperationException(
"Map key type must be binary (UTF8): " + keyType);
}
Type valueType = mapKeyValType.getType(1);
return createHiveMap(convertField(keyType), convertField(valueType));
case ENUM:
case UTF8:
return "string";
case MAP_KEY_VALUE:
// MAP_KEY_VALUE was supposed to be used to annotate key and
// value group levels in a
// MAP. However, that is always implied by the structure of
// MAP. Hence, PARQUET-113
// dropped the requirement for having MAP_KEY_VALUE.
default:
throw new UnsupportedOperationException(
"Cannot convert Parquet type " + parquetType);
}
} else {
// if no original type then it's a record
return createHiveStruct(parquetGroupType.getFields());
@Override
public String convertINT64(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "bigint";
}
}
}
/**
* Return a 'struct' Hive schema from a list of Parquet fields
*
* @param parquetFields : list of parquet fields
* @return : Equivalent 'struct' Hive schema
*/
private static String createHiveStruct(List<Type> parquetFields) {
StringBuilder struct = new StringBuilder();
struct.append("STRUCT< ");
for (Type field : parquetFields) {
//TODO: struct field name is only translated to support special char($)
//We will need to extend it to other collection type
struct.append(hiveCompatibleFieldName(field.getName(), true)).append(" : ");
struct.append(convertField(field)).append(", ");
}
struct.delete(struct.length() - 2, struct.length()); // Remove the last
// ", "
struct.append(">");
String finalStr = struct.toString();
// Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema.
// HDrone sync should not fail because of this.
finalStr = finalStr.replaceAll("-", "_");
return finalStr;
}
private static String hiveCompatibleFieldName(String fieldName, boolean isNested) {
String result = fieldName;
if (isNested) {
result = ColumnNameXLator.translateNestedColumn(fieldName);
}
return tickSurround(result);
}
private static String tickSurround(String result) {
if (!result.startsWith("`")) {
result = "`" + result;
}
if (!result.endsWith("`")) {
result = result + "`";
}
return result;
}
/**
* Create a 'Map' schema from Parquet map field
*
* @param keyType
* @param valueType
* @return
*/
private static String createHiveMap(String keyType, String valueType) {
return "MAP< " + keyType + ", " + valueType + ">";
}
/**
* Create an Array Hive schema from equivalent parquet list type
*
* @param elementType
* @param elementName
* @return
*/
private static String createHiveArray(Type elementType, String elementName) {
StringBuilder array = new StringBuilder();
array.append("ARRAY< ");
if (elementType.isPrimitive()) {
array.append(convertField(elementType));
} else {
final GroupType groupType = elementType.asGroupType();
final List<Type> groupFields = groupType.getFields();
if (groupFields.size() > 1 || (groupFields.size() == 1 && (
elementType.getName().equals("array") || elementType.getName()
.equals(elementName + "_tuple")))) {
array.append(convertField(elementType));
} else {
array.append(convertField(groupType.getFields().get(0)));
@Override
public String convertINT96(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "timestamp-millis";
}
@Override
public String convertFLOAT(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "float";
}
@Override
public String convertDOUBLE(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "double";
}
@Override
public String convertFIXED_LEN_BYTE_ARRAY(
PrimitiveType.PrimitiveTypeName primitiveTypeName) {
return "binary";
}
@Override
public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
if (originalType == OriginalType.UTF8
|| originalType == OriginalType.ENUM) {
return "string";
} else {
return "binary";
}
}
});
} else {
GroupType parquetGroupType = parquetType.asGroupType();
OriginalType originalType = parquetGroupType.getOriginalType();
if (originalType != null) {
switch (originalType) {
case LIST:
if (parquetGroupType.getFieldCount() != 1) {
throw new UnsupportedOperationException(
"Invalid list type " + parquetGroupType);
}
Type elementType = parquetGroupType.getType(0);
if (!elementType.isRepetition(Type.Repetition.REPEATED)) {
throw new UnsupportedOperationException(
"Invalid list type " + parquetGroupType);
}
return createHiveArray(elementType, parquetGroupType.getName());
case MAP:
if (parquetGroupType.getFieldCount() != 1 || parquetGroupType.getType(0)
.isPrimitive()) {
throw new UnsupportedOperationException(
"Invalid map type " + parquetGroupType);
}
GroupType mapKeyValType = parquetGroupType.getType(0).asGroupType();
if (!mapKeyValType.isRepetition(Type.Repetition.REPEATED) ||
!mapKeyValType.getOriginalType().equals(OriginalType.MAP_KEY_VALUE) ||
mapKeyValType.getFieldCount() != 2) {
throw new UnsupportedOperationException(
"Invalid map type " + parquetGroupType);
}
Type keyType = mapKeyValType.getType(0);
if (!keyType.isPrimitive() ||
!keyType.asPrimitiveType().getPrimitiveTypeName()
.equals(PrimitiveType.PrimitiveTypeName.BINARY) ||
!keyType.getOriginalType().equals(OriginalType.UTF8)) {
throw new UnsupportedOperationException(
"Map key type must be binary (UTF8): " + keyType);
}
Type valueType = mapKeyValType.getType(1);
return createHiveMap(convertField(keyType), convertField(valueType));
case ENUM:
case UTF8:
return "string";
case MAP_KEY_VALUE:
// MAP_KEY_VALUE was supposed to be used to annotate key and
// value group levels in a
// MAP. However, that is always implied by the structure of
// MAP. Hence, PARQUET-113
// dropped the requirement for having MAP_KEY_VALUE.
default:
throw new UnsupportedOperationException(
"Cannot convert Parquet type " + parquetType);
}
array.append(">");
return array.toString();
} else {
// if no original type then it's a record
return createHiveStruct(parquetGroupType.getFields());
}
}
}
/**
* Return a 'struct' Hive schema from a list of Parquet fields
*
* @param parquetFields : list of parquet fields
* @return : Equivalent 'struct' Hive schema
*/
private static String createHiveStruct(List<Type> parquetFields) {
StringBuilder struct = new StringBuilder();
struct.append("STRUCT< ");
for (Type field : parquetFields) {
//TODO: struct field name is only translated to support special char($)
//We will need to extend it to other collection type
struct.append(hiveCompatibleFieldName(field.getName(), true)).append(" : ");
struct.append(convertField(field)).append(", ");
}
struct.delete(struct.length() - 2, struct.length()); // Remove the last
// ", "
struct.append(">");
String finalStr = struct.toString();
// Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema.
// HDrone sync should not fail because of this.
finalStr = finalStr.replaceAll("-", "_");
return finalStr;
}
private static String hiveCompatibleFieldName(String fieldName, boolean isNested) {
String result = fieldName;
if (isNested) {
result = ColumnNameXLator.translateNestedColumn(fieldName);
}
return tickSurround(result);
}
private static String tickSurround(String result) {
if (!result.startsWith("`")) {
result = "`" + result;
}
if (!result.endsWith("`")) {
result = result + "`";
}
return result;
}
/**
* Create a 'Map' schema from Parquet map field
*/
private static String createHiveMap(String keyType, String valueType) {
return "MAP< " + keyType + ", " + valueType + ">";
}
/**
* Create an Array Hive schema from equivalent parquet list type
*/
private static String createHiveArray(Type elementType, String elementName) {
StringBuilder array = new StringBuilder();
array.append("ARRAY< ");
if (elementType.isPrimitive()) {
array.append(convertField(elementType));
} else {
final GroupType groupType = elementType.asGroupType();
final List<Type> groupFields = groupType.getFields();
if (groupFields.size() > 1 || (groupFields.size() == 1 && (
elementType.getName().equals("array") || elementType.getName()
.equals(elementName + "_tuple")))) {
array.append(convertField(elementType));
} else {
array.append(convertField(groupType.getFields().get(0)));
}
}
array.append(">");
return array.toString();
}
public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) {
if (prevType == null || prevType.trim().isEmpty() ||
newType == null || newType.trim().isEmpty()) {
return false;
}
prevType = prevType.toLowerCase();
newType = newType.toLowerCase();
if (prevType.equals(newType)) {
return true;
} else if (prevType.equalsIgnoreCase("int") && newType.equalsIgnoreCase("bigint")) {
return true;
} else if (prevType.equalsIgnoreCase("float") && newType.equalsIgnoreCase("double")) {
return true;
} else if (prevType.contains("struct") && newType.toLowerCase().contains("struct")) {
return true;
}
return false;
}
public static String generateSchemaString(MessageType storageSchema) throws IOException {
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
StringBuilder columns = new StringBuilder();
for (Map.Entry<String, String> hiveSchemaEntry : hiveSchema.entrySet()) {
columns.append(hiveSchemaEntry.getKey()).append(" ");
columns.append(hiveSchemaEntry.getValue()).append(", ");
}
// Remove the last ", "
columns.delete(columns.length() - 2, columns.length());
return columns.toString();
}
public static String generateCreateDDL(MessageType storageSchema,
HiveSyncConfig config, String inputFormatClass,
String outputFormatClass, String serdeClass) throws IOException {
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
String columns = generateSchemaString(storageSchema);
StringBuilder partitionFields = new StringBuilder();
for (String partitionKey : config.partitionFields) {
partitionFields.append(partitionKey).append(" ")
.append(getPartitionKeyType(hiveSchema, partitionKey));
}
public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) {
if (prevType == null || prevType.trim().isEmpty() ||
newType == null || newType.trim().isEmpty()) {
return false;
}
prevType = prevType.toLowerCase();
newType = newType.toLowerCase();
if (prevType.equals(newType)) {
return true;
} else if (prevType.equalsIgnoreCase("int") && newType.equalsIgnoreCase("bigint")) {
return true;
} else if (prevType.equalsIgnoreCase("float") && newType.equalsIgnoreCase("double")) {
return true;
} else if (prevType.contains("struct") && newType.toLowerCase().contains("struct")) {
return true;
}
return false;
StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS ");
sb = sb.append(config.databaseName).append(".").append(config.tableName);
sb = sb.append("( ").append(columns).append(")");
if (!config.partitionFields.isEmpty()) {
sb = sb.append(" PARTITIONED BY (").append(partitionFields).append(")");
}
sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'");
sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'");
sb = sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '")
.append(config.basePath).append("'");
return sb.toString();
}
public static String generateSchemaString(MessageType storageSchema) throws IOException {
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
StringBuilder columns = new StringBuilder();
for (Map.Entry<String, String> hiveSchemaEntry : hiveSchema.entrySet()) {
columns.append(hiveSchemaEntry.getKey()).append(" ");
columns.append(hiveSchemaEntry.getValue()).append(", ");
}
// Remove the last ", "
columns.delete(columns.length() - 2, columns.length());
return columns.toString();
}
public static String generateCreateDDL(MessageType storageSchema,
HiveSyncConfig config, String inputFormatClass,
String outputFormatClass, String serdeClass) throws IOException {
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema);
String columns = generateSchemaString(storageSchema);
StringBuilder partitionFields = new StringBuilder();
for (String partitionKey : config.partitionFields) {
partitionFields.append(partitionKey).append(" ")
.append(getPartitionKeyType(hiveSchema, partitionKey));
}
StringBuilder sb = new StringBuilder("CREATE EXTERNAL TABLE IF NOT EXISTS ");
sb = sb.append(config.databaseName).append(".").append(config.tableName);
sb = sb.append("( ").append(columns).append(")");
if (!config.partitionFields.isEmpty()) {
sb = sb.append(" PARTITIONED BY (").append(partitionFields).append(")");
}
sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'");
sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'");
sb = sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '")
.append(config.basePath).append("'");
return sb.toString();
}
private static String getPartitionKeyType(Map<String, String> hiveSchema, String partitionKey) {
if (hiveSchema.containsKey(partitionKey)) {
return hiveSchema.get(partitionKey);
}
// Default the unknown partition fields to be String
// TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that
return "String";
private static String getPartitionKeyType(Map<String, String> hiveSchema, String partitionKey) {
if (hiveSchema.containsKey(partitionKey)) {
return hiveSchema.get(partitionKey);
}
// Default the unknown partition fields to be String
// TODO - all partition fields should be part of the schema. datestr is treated as special. Dont do that
return "String";
}
}