Moving depedencies off cdh to apache + Hive2 support
- Tests redone in the process - Main changes are to RealtimeRecordReader and how it treats maps/arrays - Make hive sync work with Hive 1/2 and CDH environments - Fixes to make corner cases for Hive queries - Spark Hive integration - Working version across Apache and CDH versions - Known Issue - https://github.com/uber/hudi/issues/439
This commit is contained in:
committed by
vinoth chandar
parent
2b1af18941
commit
a5359662be
@@ -69,4 +69,20 @@ public class HiveSyncConfig implements Serializable {
|
||||
|
||||
@Parameter(names = {"--help", "-h"}, help = true)
|
||||
public Boolean help = false;
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HiveSyncConfig{"
|
||||
+ "databaseName='" + databaseName + '\''
|
||||
+ ", tableName='" + tableName + '\''
|
||||
+ ", hiveUser='" + hiveUser + '\''
|
||||
+ ", hivePass='" + hivePass + '\''
|
||||
+ ", jdbcUrl='" + jdbcUrl + '\''
|
||||
+ ", basePath='" + basePath + '\''
|
||||
+ ", partitionFields=" + partitionFields
|
||||
+ ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\''
|
||||
+ ", assumeDatePartitioning=" + assumeDatePartitioning
|
||||
+ ", help=" + help
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,6 +22,7 @@ import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieFileFormat;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
@@ -34,6 +35,7 @@ import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.Driver;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
@@ -43,6 +45,9 @@ import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.commons.dbcp.ConnectionFactory;
|
||||
import org.apache.commons.dbcp.DriverConnectionFactory;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
@@ -185,8 +190,7 @@ public class HoodieHiveClient {
|
||||
|
||||
String fullPartitionPath = new Path(syncConfig.basePath, partition).toString();
|
||||
String changePartition =
|
||||
alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '"
|
||||
+ "hdfs://nameservice1" + fullPartitionPath + "'";
|
||||
alterTable + " PARTITION (" + partBuilder.toString() + ") SET LOCATION '" + fullPartitionPath + "'";
|
||||
changePartitions.add(changePartition);
|
||||
}
|
||||
return changePartitions;
|
||||
@@ -234,7 +238,7 @@ public class HoodieHiveClient {
|
||||
|
||||
void updateTableDefinition(MessageType newSchema) {
|
||||
try {
|
||||
String newSchemaStr = SchemaUtil.generateSchemaString(newSchema);
|
||||
String newSchemaStr = SchemaUtil.generateSchemaString(newSchema, syncConfig.partitionFields);
|
||||
// Cascade clause should not be present for non-partitioned tables
|
||||
String cascadeClause = syncConfig.partitionFields.size() > 0 ? " cascade" : "";
|
||||
StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append("`")
|
||||
@@ -242,7 +246,7 @@ public class HoodieHiveClient {
|
||||
.append(syncConfig.tableName).append("`")
|
||||
.append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )")
|
||||
.append(cascadeClause);
|
||||
LOG.info("Creating table with " + sqlBuilder);
|
||||
LOG.info("Updating table definition with " + sqlBuilder);
|
||||
updateHiveSQL(sqlBuilder.toString());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException("Failed to update table for " + syncConfig.tableName, e);
|
||||
@@ -311,7 +315,8 @@ public class HoodieHiveClient {
|
||||
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()));
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()
|
||||
+ ", Metadata :" + commitMetadata));
|
||||
return readSchemaFromDataFile(new Path(filePath));
|
||||
case MERGE_ON_READ:
|
||||
// If this is MOR, depending on whether the latest commit is a delta commit or
|
||||
@@ -340,12 +345,31 @@ public class HoodieHiveClient {
|
||||
// read from the log file wrote
|
||||
commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
activeTimeline.getInstantDetails(lastDeltaInstant).get(), HoodieCommitMetadata.class);
|
||||
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 " + lastDeltaInstant
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath));
|
||||
Pair<String, HoodieFileFormat> filePathWithFormat =
|
||||
commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
|
||||
.stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION))
|
||||
.findAny().map(f -> Pair.of(f, HoodieFileFormat.HOODIE_LOG))
|
||||
.orElseGet(() -> {
|
||||
// No Log files in Delta-Commit. Check if there are any parquet files
|
||||
return commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream()
|
||||
.filter(s -> s.contains((metaClient.getTableConfig().getROFileFormat().getFileExtension())))
|
||||
.findAny()
|
||||
.map(f -> Pair.of(f, HoodieFileFormat.PARQUET)).orElseThrow(() -> {
|
||||
return new IllegalArgumentException(
|
||||
"Could not find any data file written for commit " + lastDeltaInstant
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()
|
||||
+ ", CommitMetadata :" + commitMetadata);
|
||||
});
|
||||
});
|
||||
switch (filePathWithFormat.getRight()) {
|
||||
case HOODIE_LOG:
|
||||
return readSchemaFromLogFile(lastCompactionCommit, new Path(filePathWithFormat.getLeft()));
|
||||
case PARQUET:
|
||||
return readSchemaFromDataFile(new Path(filePathWithFormat.getLeft()));
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown file format :" + filePathWithFormat.getRight()
|
||||
+ " for file " + filePathWithFormat.getLeft());
|
||||
}
|
||||
} else {
|
||||
return readSchemaFromLastCompaction(lastCompactionCommit);
|
||||
}
|
||||
@@ -442,14 +466,15 @@ public class HoodieHiveClient {
|
||||
|
||||
private void createHiveConnection() {
|
||||
if (connection == null) {
|
||||
BasicDataSource ds = new BasicDataSource();
|
||||
ds.setDriverClassName(driverName);
|
||||
BasicDataSource ds = new HiveDataSource();
|
||||
ds.setDriverClassName(HiveDriver.class.getCanonicalName());
|
||||
ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
|
||||
ds.setUsername(syncConfig.hiveUser);
|
||||
ds.setPassword(syncConfig.hivePass);
|
||||
LOG.info("Getting Hive Connection from Datasource " + ds);
|
||||
try {
|
||||
this.connection = ds.getConnection();
|
||||
LOG.info("Successfully got Hive Connection from Datasource " + ds);
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Cannot create hive connection " + getHiveJdbcUrlWithDefaultDBName(), e);
|
||||
@@ -589,4 +614,54 @@ public class HoodieHiveClient {
|
||||
return new PartitionEvent(PartitionEventType.UPDATE, storagePartition);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* There is a bug in BasicDataSource implementation (dbcp-1.4) which does not allow custom version of Driver (needed
|
||||
* to talk to older version of HiveServer2 including CDH-5x). This is fixed in dbcp-2x but we are using dbcp1.4.
|
||||
* Adding a workaround here. TODO: varadarb We need to investigate moving to dbcp-2x
|
||||
*/
|
||||
protected class HiveDataSource extends BasicDataSource {
|
||||
|
||||
protected ConnectionFactory createConnectionFactory() throws SQLException {
|
||||
try {
|
||||
Driver driver = HiveDriver.class.newInstance();
|
||||
// Can't test without a validationQuery
|
||||
if (validationQuery == null) {
|
||||
setTestOnBorrow(false);
|
||||
setTestOnReturn(false);
|
||||
setTestWhileIdle(false);
|
||||
}
|
||||
|
||||
// Set up the driver connection factory we will use
|
||||
String user = username;
|
||||
if (user != null) {
|
||||
connectionProperties.put("user", user);
|
||||
} else {
|
||||
log("DBCP DataSource configured without a 'username'");
|
||||
}
|
||||
|
||||
String pwd = password;
|
||||
if (pwd != null) {
|
||||
connectionProperties.put("password", pwd);
|
||||
} else {
|
||||
log("DBCP DataSource configured without a 'password'");
|
||||
}
|
||||
|
||||
ConnectionFactory driverConnectionFactory = new DriverConnectionFactory(driver, url, connectionProperties);
|
||||
return driverConnectionFactory;
|
||||
} catch (Throwable x) {
|
||||
LOG.warn("Got exception trying to instantiate connection factory. Trying default instantiation", x);
|
||||
return super.createConnectionFactory();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "HiveDataSource{"
|
||||
+ "driverClassName='" + driverClassName + '\''
|
||||
+ ", driverClassLoader=" + driverClassLoader
|
||||
+ ", url='" + url + '\''
|
||||
+ '}';
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,9 +27,11 @@ 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.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.slf4j.Logger;
|
||||
@@ -322,6 +324,14 @@ public class SchemaUtil {
|
||||
return result;
|
||||
}
|
||||
|
||||
private static String removeSurroundingTick(String result) {
|
||||
if (result.startsWith("`") && result.endsWith("`")) {
|
||||
result = result.substring(1, result.length() - 1);
|
||||
}
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a 'Map' schema from Parquet map field
|
||||
*/
|
||||
@@ -372,11 +382,17 @@ public class SchemaUtil {
|
||||
}
|
||||
|
||||
public static String generateSchemaString(MessageType storageSchema) throws IOException {
|
||||
return generateSchemaString(storageSchema, new ArrayList<>());
|
||||
}
|
||||
|
||||
public static String generateSchemaString(MessageType storageSchema, List<String> colsToSkip) 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(", ");
|
||||
if (!colsToSkip.contains(removeSurroundingTick(hiveSchemaEntry.getKey()))) {
|
||||
columns.append(hiveSchemaEntry.getKey()).append(" ");
|
||||
columns.append(hiveSchemaEntry.getValue()).append(", ");
|
||||
}
|
||||
}
|
||||
// Remove the last ", "
|
||||
columns.delete(columns.length() - 2, columns.length());
|
||||
@@ -386,19 +402,20 @@ public class SchemaUtil {
|
||||
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);
|
||||
String columns = generateSchemaString(storageSchema, config.partitionFields);
|
||||
|
||||
StringBuilder partitionFields = new StringBuilder();
|
||||
List<String> partitionFields = new ArrayList<>();
|
||||
for (String partitionKey : config.partitionFields) {
|
||||
partitionFields.append(partitionKey).append(" ")
|
||||
.append(getPartitionKeyType(hiveSchema, partitionKey));
|
||||
partitionFields.add(new StringBuilder().append(partitionKey).append(" ")
|
||||
.append(getPartitionKeyType(hiveSchema, partitionKey)).toString());
|
||||
}
|
||||
|
||||
String paritionsStr = partitionFields.stream().collect(Collectors.joining(","));
|
||||
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(" PARTITIONED BY (").append(paritionsStr).append(")");
|
||||
}
|
||||
sb = sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'");
|
||||
sb = sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'");
|
||||
|
||||
Reference in New Issue
Block a user