1
0

HUDI-123 Rename code packages/constants to org.apache.hudi (#830)

- Rename com.uber.hoodie to org.apache.hudi
- Flag to pass com.uber.hoodie Input formats for hoodie-sync
- Works with HUDI demo. 
- Also tested for backwards compatibility with datasets built by com.uber.hoodie packages
- Migration guide : https://cwiki.apache.org/confluence/display/HUDI/Migration+Guide+From+com.uber.hoodie+to+org.apache.hudi
This commit is contained in:
Balaji Varadarajan
2019-08-11 17:48:17 -07:00
committed by vinoth chandar
parent 722b6be04a
commit a4f9d7575f
546 changed files with 3858 additions and 3562 deletions

View File

@@ -0,0 +1,46 @@
<!--
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.
-->
<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.3 http://maven.apache.org/xsd/assembly-1.1.3.xsd">
<id>jar-with-dependencies</id>
<formats>
<format>jar</format>
</formats>
<includeBaseDirectory>false</includeBaseDirectory>
<dependencySets>
<dependencySet>
<outputDirectory>/</outputDirectory>
<unpack>true</unpack>
<scope>runtime</scope>
<excludes>
<exclude>junit:junit</exclude>
<exclude>com.google.code.findbugs:*</exclude>
<exclude>org.apache.hbase:*</exclude>
</excludes>
</dependencySet>
<dependencySet>
<unpack>true</unpack>
<scope>provided</scope>
</dependencySet>
</dependencySets>
</assembly>

View File

@@ -0,0 +1,110 @@
/*
* 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 org.apache.hudi.hive;
import com.beust.jcommander.Parameter;
import java.io.Serializable;
import java.util.ArrayList;
import java.util.List;
/**
* Configs needed to sync data into Hive.
*/
public class HiveSyncConfig implements Serializable {
@Parameter(names = {
"--database"}, description = "name of the target database in Hive", required = true)
public String databaseName;
@Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true)
public String tableName;
@Parameter(names = {"--user"}, description = "Hive username", required = true)
public String hiveUser;
@Parameter(names = {"--pass"}, description = "Hive password", required = true)
public String hivePass;
@Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url", required = true)
public String jdbcUrl;
@Parameter(names = {
"--base-path"}, description = "Basepath of hoodie dataset to sync", required = true)
public String basePath;
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by",
required = false)
public List<String> partitionFields = new ArrayList<>();
@Parameter(names = "-partition-value-extractor", description = "Class which implements "
+ "PartitionValueExtractor "
+ "to extract the partition "
+ "values from HDFS path")
public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class
.getName();
@Parameter(names = {
"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"
+ " exists to support "
+ "backward compatibility. If"
+ " you use hoodie 0.3.x, do "
+ "not set this parameter")
public Boolean assumeDatePartitioning = false;
@Parameter(names = {
"--use-pre-apache-input-format"}, description = "Use InputFormat under com.uber.hoodie package "
+ "instead of org.apache.hudi package. Use this when you are in the process of migrating from "
+ "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to "
+ "org.apache.hudi input format.")
public Boolean usePreApacheInputFormat = false;
@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
+ '}';
}
public static HiveSyncConfig copy(HiveSyncConfig cfg) {
HiveSyncConfig newConfig = new HiveSyncConfig();
newConfig.basePath = cfg.basePath;
newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning;
newConfig.databaseName = cfg.databaseName;
newConfig.hivePass = cfg.hivePass;
newConfig.hiveUser = cfg.hiveUser;
newConfig.partitionFields = cfg.partitionFields;
newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass;
newConfig.jdbcUrl = cfg.jdbcUrl;
newConfig.tableName = cfg.tableName;
newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat;
return newConfig;
}
}

View File

@@ -0,0 +1,197 @@
/*
* 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 org.apache.hudi.hive;
import com.beust.jcommander.JCommander;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.hadoop.HoodieInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieRealtimeInputFormat;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import org.apache.hudi.hive.util.SchemaUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.schema.MessageType;
/**
* 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]
* <p>
* 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 {
private static final Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class);
private final HoodieHiveClient hoodieHiveClient;
public static final String SUFFIX_REALTIME_TABLE = "_rt";
private final HiveSyncConfig cfg;
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs);
this.cfg = cfg;
}
public void syncHoodieTable() throws ClassNotFoundException {
switch (hoodieHiveClient.getTableType()) {
case COPY_ON_WRITE:
syncHoodieTable(false);
break;
case MERGE_ON_READ:
//sync a RO table for MOR
syncHoodieTable(false);
String originalTableName = cfg.tableName;
//TODO : Make realtime table registration optional using a config param
cfg.tableName = cfg.tableName + SUFFIX_REALTIME_TABLE;
//sync a RT table for MOR
syncHoodieTable(true);
cfg.tableName = originalTableName;
break;
default:
LOG.error("Unknown table type " + hoodieHiveClient.getTableType());
throw new InvalidDatasetException(hoodieHiveClient.getBasePath());
}
hoodieHiveClient.close();
}
private void syncHoodieTable(boolean isRealTime) throws ClassNotFoundException {
LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path "
+ hoodieHiveClient.getBasePath() + " of type " + hoodieHiveClient.getTableType());
// Check if the necessary table exists
boolean tableExists = hoodieHiveClient.doesTableExist();
// Get the parquet schema for this dataset looking at the latest commit
MessageType schema = hoodieHiveClient.getDataSchema();
// Sync schema if needed
syncSchema(tableExists, isRealTime, schema);
LOG.info("Schema sync complete. Syncing partitions for " + cfg.tableName);
// Get the last time we successfully synced partitions
Option<String> lastCommitTimeSynced = Option.empty();
if (tableExists) {
lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced();
}
LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null"));
List<String> writtenPartitionsSince = hoodieHiveClient
.getPartitionsWrittenToSince(lastCommitTimeSynced);
LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size());
// Sync the partitions if needed
syncPartitions(writtenPartitionsSince);
hoodieHiveClient.updateLastCommitTimeSynced();
LOG.info("Sync complete for " + cfg.tableName);
}
/**
* Get the latest schema from the last commit and check if its in sync with the hive table schema.
* If not, evolves the table schema.
*
* @param tableExists - does table exist
* @param schema - extracted schema
*/
private void syncSchema(boolean tableExists, boolean isRealTime, MessageType schema) throws ClassNotFoundException {
// Check and sync schema
if (!tableExists) {
LOG.info("Table " + cfg.tableName + " is not found. Creating it");
if (!isRealTime) {
// TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default
// for now)
String inputFormatClassName =
cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.HoodieInputFormat.class.getName()
: HoodieInputFormat.class.getName();
hoodieHiveClient.createTable(schema, inputFormatClassName,
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
String inputFormatClassName =
cfg.usePreApacheInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName()
: HoodieRealtimeInputFormat.class.getName();
hoodieHiveClient.createTable(schema, inputFormatClassName,
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
}
} else {
// Check if the dataset schema has evolved
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema();
SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema,
cfg.partitionFields);
if (!schemaDiff.isEmpty()) {
LOG.info("Schema difference found for " + cfg.tableName);
hoodieHiveClient.updateTableDefinition(schema);
} else {
LOG.info("No Schema difference for " + cfg.tableName);
}
}
}
/**
* Syncs the list of storage parititions passed in (checks if the partition is in hive, if not
* adds it or if the partition path does not match, it updates the partition path)
*/
private void syncPartitions(List<String> writtenPartitionsSince) {
try {
List<Partition> hivePartitions = hoodieHiveClient.scanTablePartitions();
List<PartitionEvent> partitionEvents = hoodieHiveClient.getPartitionEvents(hivePartitions,
writtenPartitionsSince);
List<String> newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD);
LOG.info("New Partitions " + newPartitions);
hoodieHiveClient.addPartitionsToTable(newPartitions);
List<String> updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE);
LOG.info("Changed Partitions " + updatePartitions);
hoodieHiveClient.updatePartitionsToTable(updatePartitions);
} catch (Exception e) {
throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName, e);
}
}
private List<String> filterPartitions(List<PartitionEvent> events, PartitionEventType eventType) {
return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition)
.collect(Collectors.toList());
}
public static void main(String[] args) throws Exception {
// parse the params
final HiveSyncConfig cfg = new HiveSyncConfig();
JCommander cmd = new JCommander(cfg, args);
if (cfg.help || args.length == 0) {
cmd.usage();
System.exit(1);
}
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
HiveConf hiveConf = new HiveConf();
hiveConf.addResource(fs.getConf());
new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable();
}
}

View File

@@ -0,0 +1,680 @@
/*
* 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 org.apache.hudi.hive;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
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;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
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.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hive.jdbc.HiveDriver;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.InvalidDatasetException;
import org.apache.hudi.hive.util.SchemaUtil;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.format.converter.ParquetMetadataConverter;
import parquet.hadoop.ParquetFileReader;
import parquet.hadoop.metadata.ParquetMetadata;
import parquet.schema.MessageType;
@SuppressWarnings("ConstantConditions")
public class HoodieHiveClient {
private static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync";
// Make sure we have the hive JDBC driver in classpath
private static String driverName = HiveDriver.class.getName();
static {
try {
Class.forName(driverName);
} catch (ClassNotFoundException e) {
throw new IllegalStateException("Could not find " + driverName + " in classpath. ", e);
}
}
private static Logger LOG = LoggerFactory.getLogger(HoodieHiveClient.class);
private final HoodieTableMetaClient metaClient;
private final HoodieTableType tableType;
private final PartitionValueExtractor partitionValueExtractor;
private HiveMetaStoreClient client;
private HiveSyncConfig syncConfig;
private FileSystem fs;
private Connection connection;
private HoodieTimeline activeTimeline;
public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
this.syncConfig = cfg;
this.fs = fs;
this.metaClient = new HoodieTableMetaClient(fs.getConf(), cfg.basePath, true);
this.tableType = metaClient.getTableType();
LOG.info("Creating hive connection " + cfg.jdbcUrl);
createHiveConnection();
try {
this.client = new HiveMetaStoreClient(configuration);
} catch (MetaException e) {
throw new HoodieHiveSyncException("Failed to create HiveMetaStoreClient", e);
}
try {
this.partitionValueExtractor = (PartitionValueExtractor) Class.forName(
cfg.partitionValueExtractorClass).newInstance();
} catch (Exception e) {
throw new HoodieHiveSyncException(
"Failed to initialize PartitionValueExtractor class " + cfg.partitionValueExtractorClass,
e);
}
activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
}
public HoodieTimeline getActiveTimeline() {
return activeTimeline;
}
/**
* Add the (NEW) partitons to the table
*/
void addPartitionsToTable(List<String> partitionsToAdd) {
if (partitionsToAdd.isEmpty()) {
LOG.info("No partitions to add for " + syncConfig.tableName);
return;
}
LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + syncConfig.tableName);
String sql = constructAddPartitions(partitionsToAdd);
updateHiveSQL(sql);
}
/**
* Partition path has changed - update the path for te following partitions
*/
void updatePartitionsToTable(List<String> changedPartitions) {
if (changedPartitions.isEmpty()) {
LOG.info("No partitions to change for " + syncConfig.tableName);
return;
}
LOG.info("Changing partitions " + changedPartitions.size() + " on " + syncConfig.tableName);
List<String> sqls = constructChangePartitions(changedPartitions);
for (String sql : sqls) {
updateHiveSQL(sql);
}
}
private String constructAddPartitions(List<String> partitions) {
StringBuilder alterSQL = new StringBuilder("ALTER TABLE ");
alterSQL.append(syncConfig.databaseName).append(".").append(syncConfig.tableName)
.append(" ADD IF NOT EXISTS ");
for (String partition : partitions) {
String partitionClause = getPartitionClause(partition);
String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString();
alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '")
.append(fullPartitionPath).append("' ");
}
return alterSQL.toString();
}
/**
* Generate Hive Partition from partition values
* @param partition Partition path
* @return
*/
private String getPartitionClause(String partition) {
List<String> partitionValues = partitionValueExtractor
.extractPartitionValuesInPath(partition);
Preconditions.checkArgument(syncConfig.partitionFields.size() == partitionValues.size(),
"Partition key parts " + syncConfig.partitionFields
+ " does not match with partition values " + partitionValues
+ ". Check partition strategy. ");
List<String> partBuilder = new ArrayList<>();
for (int i = 0; i < syncConfig.partitionFields.size(); i++) {
partBuilder.add(syncConfig.partitionFields.get(i) + "=" + "'" + partitionValues.get(i) + "'");
}
return partBuilder.stream().collect(Collectors.joining(","));
}
private List<String> constructChangePartitions(List<String> partitions) {
List<String> changePartitions = Lists.newArrayList();
// Hive 2.x doesn't like db.table name for operations, hence we need to change to using the database first
String useDatabase = "USE " + syncConfig.databaseName;
changePartitions.add(useDatabase);
String alterTable = "ALTER TABLE " + syncConfig.tableName;
for (String partition : partitions) {
String partitionClause = getPartitionClause(partition);
String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString();
String changePartition =
alterTable + " PARTITION (" + partitionClause + ") SET LOCATION '" + fullPartitionPath + "'";
changePartitions.add(changePartition);
}
return changePartitions;
}
/**
* 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) {
Map<String, String> paths = Maps.newHashMap();
for (Partition tablePartition : tablePartitions) {
List<String> hivePartitionValues = tablePartition.getValues();
Collections.sort(hivePartitionValues);
String fullTablePartitionPath = Path.getPathWithoutSchemeAndAuthority(
new Path(tablePartition.getSd().getLocation())).toUri().getPath();
paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath);
}
List<PartitionEvent> events = Lists.newArrayList();
for (String storagePartition : partitionStoragePartitions) {
String fullStoragePartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, storagePartition).toString();
// Check if the partition values or if hdfs path is the same
List<String> storagePartitionValues = partitionValueExtractor
.extractPartitionValuesInPath(storagePartition);
Collections.sort(storagePartitionValues);
if (!storagePartitionValues.isEmpty()) {
String storageValue = String.join(", ", storagePartitionValues);
if (!paths.containsKey(storageValue)) {
events.add(PartitionEvent.newPartitionAddEvent(storagePartition));
} else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) {
events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition));
}
}
}
return events;
}
/**
* Scan table partitions
*/
public List<Partition> scanTablePartitions() throws TException {
return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1);
}
void updateTableDefinition(MessageType newSchema) {
try {
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("`")
.append(syncConfig.databaseName).append(".")
.append(syncConfig.tableName).append("`")
.append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )")
.append(cascadeClause);
LOG.info("Updating table definition with " + sqlBuilder);
updateHiveSQL(sqlBuilder.toString());
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to update table for " + syncConfig.tableName, e);
}
}
void createTable(MessageType storageSchema, String inputFormatClass, String outputFormatClass,
String serdeClass) {
try {
String createSQLQuery = SchemaUtil
.generateCreateDDL(storageSchema, syncConfig, inputFormatClass,
outputFormatClass, serdeClass);
LOG.info("Creating table with " + createSQLQuery);
updateHiveSQL(createSQLQuery);
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to create table " + syncConfig.tableName, e);
}
}
/**
* Get the table schema
*/
public Map<String, String> getTableSchema() {
if (!doesTableExist()) {
throw new IllegalArgumentException(
"Failed to get schema for table " + syncConfig.tableName + " does not exist");
}
Map<String, String> schema = Maps.newHashMap();
ResultSet result = null;
try {
DatabaseMetaData databaseMetaData = connection.getMetaData();
result = databaseMetaData
.getColumns(null, syncConfig.databaseName, syncConfig.tableName, null);
while (result.next()) {
String columnName = result.getString(4);
String columnType = result.getString(6);
if ("DECIMAL".equals(columnType)) {
int columnSize = result.getInt("COLUMN_SIZE");
int decimalDigits = result.getInt("DECIMAL_DIGITS");
columnType += String.format("(%s,%s)", columnSize, decimalDigits);
}
schema.put(columnName, columnType);
}
return schema;
} catch (SQLException e) {
throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName,
e);
} finally {
closeQuietly(result, null);
}
}
/**
* 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
*/
@SuppressWarnings("WeakerAccess")
public MessageType getDataSchema() {
try {
switch (tableType) {
case COPY_ON_WRITE:
// If this is COW, get the last commit and read the schema from a file written in the
// last commit
HoodieInstant lastCommit = activeTimeline.lastInstant().orElseThrow(
() -> new InvalidDatasetException(syncConfig.basePath));
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
activeTimeline.getInstantDetails(lastCommit).get(), HoodieCommitMetadata.class);
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()
+ ", 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
// compaction commit
// Get a datafile written and get the schema from that file
Option<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
.getCommitTimeline()
.filterCompletedInstants()
.lastInstant();
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
Option<HoodieInstant> lastDeltaCommit;
if (lastCompactionCommit.isPresent()) {
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline()
.filterCompletedInstants()
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(),
Integer.MAX_VALUE).lastInstant();
} else {
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline()
.filterCompletedInstants().lastInstant();
}
LOG.info("Found the last delta commit " + lastDeltaCommit);
if (lastDeltaCommit.isPresent()) {
HoodieInstant lastDeltaInstant = lastDeltaCommit.get();
// read from the log file wrote
commitMetadata = HoodieCommitMetadata.fromBytes(
activeTimeline.getInstantDetails(lastDeltaInstant).get(), HoodieCommitMetadata.class);
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);
}
default:
LOG.error("Unknown table type " + tableType);
throw new InvalidDatasetException(syncConfig.basePath);
}
} catch (IOException e) {
throw new HoodieHiveSyncException("Failed to get dataset schema for " + syncConfig.tableName,
e);
}
}
/**
* Read schema from a data file from the last compaction commit done.
*/
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLastCompaction(Option<HoodieInstant> lastCompactionCommitOpt)
throws IOException {
HoodieInstant lastCompactionCommit = lastCompactionCommitOpt.orElseThrow(
() -> new HoodieHiveSyncException(
"Could not read schema from last compaction, no compaction commits found on path "
+ syncConfig.basePath));
// Read from the compacted file wrote
HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata.fromBytes(
activeTimeline.getInstantDetails(lastCompactionCommit).get(), HoodieCommitMetadata.class);
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()));
return readSchemaFromDataFile(new Path(filePath));
}
/**
* Read the schema from the log file on path
*/
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
private MessageType readSchemaFromLogFile(Option<HoodieInstant> lastCompactionCommitOpt,
Path path) throws IOException {
MessageType messageType = SchemaUtil.readSchemaFromLogFile(fs, path);
// Fall back to read the schema from last compaction
if (messageType == null) {
LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt);
return readSchemaFromLastCompaction(lastCompactionCommitOpt);
}
return messageType;
}
/**
* Read the parquet schema from a parquet File
*/
private MessageType readSchemaFromDataFile(Path parquetFilePath) throws IOException {
LOG.info("Reading schema from " + parquetFilePath);
if (!fs.exists(parquetFilePath)) {
throw new IllegalArgumentException(
"Failed to read schema from data file " + parquetFilePath + ". File does not exist.");
}
ParquetMetadata fileFooter = ParquetFileReader.readFooter(fs.getConf(), parquetFilePath,
ParquetMetadataConverter.NO_FILTER);
return fileFooter.getFileMetaData().getSchema();
}
/**
* @return true if the configured table exists
*/
public boolean doesTableExist() {
try {
return client.tableExists(syncConfig.databaseName, syncConfig.tableName);
} catch (TException e) {
throw new HoodieHiveSyncException("Failed to check if table exists " + syncConfig.tableName,
e);
}
}
/**
* Execute a update in hive metastore with this SQL
*
* @param s SQL to execute
*/
public void updateHiveSQL(String s) {
Statement stmt = null;
try {
stmt = connection.createStatement();
LOG.info("Executing SQL " + s);
stmt.execute(s);
} catch (SQLException e) {
throw new HoodieHiveSyncException("Failed in executing SQL " + s, e);
} finally {
closeQuietly(null, stmt);
}
}
private void createHiveConnection() {
if (connection == null) {
BasicDataSource ds = new HiveDataSource();
ds.setDriverClassName(HiveDriver.class.getCanonicalName());
ds.setUrl(getHiveJdbcUrlWithDefaultDBName());
if (syncConfig.hiveUser != null) {
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);
}
}
}
private String getHiveJdbcUrlWithDefaultDBName() {
String hiveJdbcUrl = syncConfig.jdbcUrl;
String urlAppend = null;
// If the hive url contains addition properties like ;transportMode=http;httpPath=hs2
if (hiveJdbcUrl.contains(";")) {
urlAppend = hiveJdbcUrl.substring(hiveJdbcUrl.indexOf(";"));
hiveJdbcUrl = hiveJdbcUrl.substring(0, hiveJdbcUrl.indexOf(";"));
}
if (!hiveJdbcUrl.endsWith("/")) {
hiveJdbcUrl = hiveJdbcUrl + "/";
}
return hiveJdbcUrl + (urlAppend == null ? "" : urlAppend);
}
private static void closeQuietly(ResultSet resultSet, Statement stmt) {
try {
if (stmt != null) {
stmt.close();
}
if (resultSet != null) {
resultSet.close();
}
} catch (SQLException e) {
LOG.error("Could not close the resultset opened ", e);
}
}
public String getBasePath() {
return metaClient.getBasePath();
}
HoodieTableType getTableType() {
return tableType;
}
public FileSystem getFs() {
return fs;
}
public Option<String> getLastCommitTimeSynced() {
// Get the last commit time from the TBLproperties
try {
Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName);
return Option.ofNullable(
database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null));
} catch (Exception e) {
throw new HoodieHiveSyncException(
"Failed to get the last commit time synced from the database", e);
}
}
public void close() {
try {
if (connection != null) {
connection.close();
}
if (client != null) {
client.close();
}
} catch (SQLException e) {
LOG.error("Could not close connection ", e);
}
}
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
List<String> getPartitionsWrittenToSince(Option<String> lastCommitTimeSynced) {
if (!lastCommitTimeSynced.isPresent()) {
LOG.info("Last commit time synced is not known, listing all partitions in " + syncConfig.basePath + ",FS :" + fs);
try {
return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath,
syncConfig.assumeDatePartitioning);
} catch (IOException e) {
throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e);
}
} else {
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get()
+ ", Getting commits since then");
HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(),
Integer.MAX_VALUE);
return timelineToSync.getInstants().map(s -> {
try {
return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get(), HoodieCommitMetadata.class);
} catch (IOException e) {
throw new HoodieIOException(
"Failed to get partitions written since " + lastCommitTimeSynced, e);
}
}).flatMap(s -> s.getPartitionToWriteStats().keySet().stream()).distinct()
.collect(Collectors.toList());
}
}
List<String> getAllTables(String db) throws Exception {
return client.getAllTables(db);
}
void updateLastCommitTimeSynced() {
// Set the last commit time from the TBLproperties
String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp();
try {
Table table = client.getTable(syncConfig.databaseName, syncConfig.tableName);
table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced);
client.alter_table(syncConfig.databaseName, syncConfig.tableName, table);
} catch (Exception e) {
throw new HoodieHiveSyncException(
"Failed to get update last commit time synced to " + lastCommitSynced, e);
}
}
/**
* Partition Event captures any partition that needs to be added or updated
*/
static class PartitionEvent {
public enum PartitionEventType {
ADD,
UPDATE
}
PartitionEventType eventType;
String storagePartition;
PartitionEvent(PartitionEventType eventType, String storagePartition) {
this.eventType = eventType;
this.storagePartition = storagePartition;
}
static PartitionEvent newPartitionAddEvent(String storagePartition) {
return new PartitionEvent(PartitionEventType.ADD, storagePartition);
}
static PartitionEvent newPartitionUpdateEvent(String storagePartition) {
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 + '\''
+ '}';
}
}
}

View File

@@ -0,0 +1,42 @@
/*
* 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 org.apache.hudi.hive;
public class HoodieHiveSyncException extends RuntimeException {
public HoodieHiveSyncException() {
super();
}
public HoodieHiveSyncException(String message) {
super(message);
}
public HoodieHiveSyncException(String message, Throwable t) {
super(message, t);
}
public HoodieHiveSyncException(Throwable t) {
super(t);
}
protected static String format(String message, Object... args) {
return String.format(String.valueOf(message), (Object[]) args);
}
}

View File

@@ -0,0 +1,44 @@
/*
* 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 org.apache.hudi.hive;
import com.google.common.base.Preconditions;
import java.util.Arrays;
import java.util.List;
import java.util.stream.Collectors;
/**
* Partition Key extractor treating each value delimited by slash as separate key.
*/
public class MultiPartKeysValueExtractor implements PartitionValueExtractor {
@Override
public List<String> extractPartitionValuesInPath(String partitionPath) {
String[] splits = partitionPath.split("/");
return Arrays.stream(splits).map(s -> {
if (s.contains("=")) {
String[] moreSplit = s.split("=");
Preconditions.checkArgument(moreSplit.length == 2,
"Partition Field (" + s + ") not in expected format");
return moreSplit[1];
}
return s;
}).collect(Collectors.toList());
}
}

View File

@@ -0,0 +1,33 @@
/*
* 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 org.apache.hudi.hive;
import java.util.ArrayList;
import java.util.List;
/**
* Extractor for Non-partitioned hive tables
*/
public class NonPartitionedExtractor implements PartitionValueExtractor {
@Override
public List<String> extractPartitionValuesInPath(String partitionPath) {
return new ArrayList<>();
}
}

View File

@@ -0,0 +1,35 @@
/*
* 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 org.apache.hudi.hive;
import java.io.Serializable;
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.
* <p>
* 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 extends Serializable {
List<String> extractPartitionValuesInPath(String partitionPath);
}

View File

@@ -0,0 +1,114 @@
/*
* 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 org.apache.hudi.hive;
import com.google.common.base.Objects;
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 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 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);
}
}
}

View File

@@ -0,0 +1,64 @@
/*
* 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 org.apache.hudi.hive;
import com.beust.jcommander.internal.Lists;
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.
* <p>
* This implementation extracts datestr=yyyy-mm-dd from path of type /yyyy/mm/dd
*/
public class SlashEncodedDayPartitionValueExtractor implements PartitionValueExtractor {
private transient DateTimeFormatter dtfOut;
public SlashEncodedDayPartitionValueExtractor() {
this.dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
}
private DateTimeFormatter getDtfOut() {
if (dtfOut == null) {
dtfOut = DateTimeFormat.forPattern("yyyy-MM-dd");
}
return dtfOut;
}
@Override
public List<String> extractPartitionValuesInPath(String partitionPath) {
// partition path is expected to be in this format yyyy/mm/dd
String[] splits = partitionPath.split("/");
if (splits.length != 3) {
throw new IllegalArgumentException(
"Partition path " + partitionPath + " is not in the form yyyy/mm/dd ");
}
// Get the partition part and remove the / as well at the end
int year = Integer.parseInt(splits[0]);
int mm = Integer.parseInt(splits[1]);
int dd = Integer.parseInt(splits[2]);
DateTime dateTime = new DateTime(year, mm, dd, 0, 0);
return Lists.newArrayList(getDtfOut().print(dateTime));
}
}

View File

@@ -0,0 +1,50 @@
/*
* 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 org.apache.hudi.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 ic = xformMap.entrySet().iterator(); ic.hasNext();
colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) {
entry = (Map.Entry) ic.next();
}
return colName;
}
public static String translateColumn(String colName) {
return colName;
}
public static String translate(String colName, boolean nestedColumn) {
return !nestedColumn ? translateColumn(colName) : translateNestedColumn(colName);
}
static {
xformMap.put("\\$", "_dollar_");
}
}

View File

@@ -0,0 +1,459 @@
/*
* 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 org.apache.hudi.hive.util;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
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.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Reader;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HoodieHiveSyncException;
import org.apache.hudi.hive.SchemaDifference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import parquet.schema.DecimalMetadata;
import parquet.schema.GroupType;
import parquet.schema.MessageType;
import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
import parquet.schema.Type;
/**
* Schema Utilities
*/
public class SchemaUtil {
private static final 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;
}
tableColumnType = tableColumnType.replaceAll("\\s+", "");
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);
}
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();
}
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]
*/
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";
}
@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());
}
}
}
/**
* 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;
}
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
*/
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 {
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()) {
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());
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, config.partitionFields);
List<String> partitionFields = new ArrayList<>();
for (String partitionKey : config.partitionFields) {
String partitionKeyWithTicks = tickSurround(partitionKey);
partitionFields.add(new StringBuilder().append(partitionKey).append(" ")
.append(getPartitionKeyType(hiveSchema, partitionKeyWithTicks)).toString());
}
String partitionsStr = 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(partitionsStr).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";
}
/**
* 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;
}
}
reader.close();
if (lastBlock != null) {
return new parquet.avro.AvroSchemaConverter().convert(lastBlock.getSchema());
}
return null;
}
}

View File

@@ -0,0 +1,614 @@
Apache License
Version 2.0, January 2004
http://www.apache.org/licenses/
TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
1. Definitions.
"License" shall mean the terms and conditions for use, reproduction,
and distribution as defined by Sections 1 through 9 of this document.
"Licensor" shall mean the copyright owner or entity authorized by
the copyright owner that is granting the License.
"Legal Entity" shall mean the union of the acting entity and all
other entities that control, are controlled by, or are under common
control with that entity. For the purposes of this definition,
"control" means (i) the power, direct or indirect, to cause the
direction or management of such entity, whether by contract or
otherwise, or (ii) ownership of fifty percent (50%) or more of the
outstanding shares, or (iii) beneficial ownership of such entity.
"You" (or "Your") shall mean an individual or Legal Entity
exercising permissions granted by this License.
"Source" form shall mean the preferred form for making modifications,
including but not limited to software source code, documentation
source, and configuration files.
"Object" form shall mean any form resulting from mechanical
transformation or translation of a Source form, including but
not limited to compiled object code, generated documentation,
and conversions to other media types.
"Work" shall mean the work of authorship, whether in Source or
Object form, made available under the License, as indicated by a
copyright notice that is included in or attached to the work
(an example is provided in the Appendix below).
"Derivative Works" shall mean any work, whether in Source or Object
form, that is based on (or derived from) the Work and for which the
editorial revisions, annotations, elaborations, or other modifications
represent, as a whole, an original work of authorship. For the purposes
of this License, Derivative Works shall not include works that remain
separable from, or merely link (or bind by name) to the interfaces of,
the Work and Derivative Works thereof.
"Contribution" shall mean any work of authorship, including
the original version of the Work and any modifications or additions
to that Work or Derivative Works thereof, that is intentionally
submitted to Licensor for inclusion in the Work by the copyright owner
or by an individual or Legal Entity authorized to submit on behalf of
the copyright owner. For the purposes of this definition, "submitted"
means any form of electronic, verbal, or written communication sent
to the Licensor or its representatives, including but not limited to
communication on electronic mailing lists, source code control systems,
and issue tracking systems that are managed by, or on behalf of, the
Licensor for the purpose of discussing and improving the Work, but
excluding communication that is conspicuously marked or otherwise
designated in writing by the copyright owner as "Not a Contribution."
"Contributor" shall mean Licensor and any individual or Legal Entity
on behalf of whom a Contribution has been received by Licensor and
subsequently incorporated within the Work.
2. Grant of Copyright License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
copyright license to reproduce, prepare Derivative Works of,
publicly display, publicly perform, sublicense, and distribute the
Work and such Derivative Works in Source or Object form.
3. Grant of Patent License. Subject to the terms and conditions of
this License, each Contributor hereby grants to You a perpetual,
worldwide, non-exclusive, no-charge, royalty-free, irrevocable
(except as stated in this section) patent license to make, have made,
use, offer to sell, sell, import, and otherwise transfer the Work,
where such license applies only to those patent claims licensable
by such Contributor that are necessarily infringed by their
Contribution(s) alone or by combination of their Contribution(s)
with the Work to which such Contribution(s) was submitted. If You
institute patent litigation against any entity (including a
cross-claim or counterclaim in a lawsuit) alleging that the Work
or a Contribution incorporated within the Work constitutes direct
or contributory patent infringement, then any patent licenses
granted to You under this License for that Work shall terminate
as of the date such litigation is filed.
4. Redistribution. You may reproduce and distribute copies of the
Work or Derivative Works thereof in any medium, with or without
modifications, and in Source or Object form, provided that You
meet the following conditions:
(a) You must give any other recipients of the Work or
Derivative Works a copy of this License; and
(b) You must cause any modified files to carry prominent notices
stating that You changed the files; and
(c) You must retain, in the Source form of any Derivative Works
that You distribute, all copyright, patent, trademark, and
attribution notices from the Source form of the Work,
excluding those notices that do not pertain to any part of
the Derivative Works; and
(d) If the Work includes a "NOTICE" text file as part of its
distribution, then any Derivative Works that You distribute must
include a readable copy of the attribution notices contained
within such NOTICE file, excluding those notices that do not
pertain to any part of the Derivative Works, in at least one
of the following places: within a NOTICE text file distributed
as part of the Derivative Works; within the Source form or
documentation, if provided along with the Derivative Works; or,
within a display generated by the Derivative Works, if and
wherever such third-party notices normally appear. The contents
of the NOTICE file are for informational purposes only and
do not modify the License. You may add Your own attribution
notices within Derivative Works that You distribute, alongside
or as an addendum to the NOTICE text from the Work, provided
that such additional attribution notices cannot be construed
as modifying the License.
You may add Your own copyright statement to Your modifications and
may provide additional or different license terms and conditions
for use, reproduction, or distribution of Your modifications, or
for any such Derivative Works as a whole, provided Your use,
reproduction, and distribution of the Work otherwise complies with
the conditions stated in this License.
5. Submission of Contributions. Unless You explicitly state otherwise,
any Contribution intentionally submitted for inclusion in the Work
by You to the Licensor shall be under the terms and conditions of
this License, without any additional terms or conditions.
Notwithstanding the above, nothing herein shall supersede or modify
the terms of any separate license agreement you may have executed
with Licensor regarding such Contributions.
6. Trademarks. This License does not grant permission to use the trade
names, trademarks, service marks, or product names of the Licensor,
except as required for reasonable and customary use in describing the
origin of the Work and reproducing the content of the NOTICE file.
7. Disclaimer of Warranty. Unless required by applicable law or
agreed to in writing, Licensor provides the Work (and each
Contributor provides its Contributions) on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
implied, including, without limitation, any warranties or conditions
of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
PARTICULAR PURPOSE. You are solely responsible for determining the
appropriateness of using or redistributing the Work and assume any
risks associated with Your exercise of permissions under this License.
8. Limitation of Liability. In no event and under no legal theory,
whether in tort (including negligence), contract, or otherwise,
unless required by applicable law (such as deliberate and grossly
negligent acts) or agreed to in writing, shall any Contributor be
liable to You for damages, including any direct, indirect, special,
incidental, or consequential damages of any character arising as a
result of this License or out of the use or inability to use the
Work (including but not limited to damages for loss of goodwill,
work stoppage, computer failure or malfunction, or any and all
other commercial damages or losses), even if such Contributor
has been advised of the possibility of such damages.
9. Accepting Warranty or Additional Liability. While redistributing
the Work or Derivative Works thereof, You may choose to offer,
and charge a fee for, acceptance of support, warranty, indemnity,
or other liability obligations and/or rights consistent with this
License. However, in accepting such obligations, You may act only
on Your own behalf and on Your sole responsibility, not on behalf
of any other Contributor, and only if You agree to indemnify,
defend, and hold each Contributor harmless for any liability
incurred by, or claims asserted against, such Contributor by reason
of your accepting any such warranty or additional liability.
END OF TERMS AND CONDITIONS
APPENDIX: How to apply the Apache License to your work.
To apply the Apache License to your work, attach the following
boilerplate notice, with the fields enclosed by brackets "[]"
replaced with your own identifying information. (Don't include
the brackets!) The text should be enclosed in the appropriate
comment syntax for the file format. We also recommend that a
file or class name and description of purpose be included on the
same "printed page" as the copyright notice for easier
identification within third-party archives.
Copyright [yyyy] [name of copyright owner]
Licensed 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.
----
This project bundles portions of the 'JQuery' project under the terms of the MIT license.
Copyright 2012 jQuery Foundation and other contributors
http://jquery.com/
Permission is hereby granted, free of charge, to any person obtaining
a copy of this software and associated documentation files (the
"Software"), to deal in the Software without restriction, including
without limitation the rights to use, copy, modify, merge, publish,
distribute, sublicense, and/or sell copies of the Software, and to
permit persons to whom the Software is furnished to do so, subject to
the following conditions:
The above copyright notice and this permission notice shall be
included in all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND,
EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF
MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND
NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION
WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.
----
This project bundles a derivative of portions of the 'Asciidoctor' project
under the terms of the MIT license.
The MIT License
Copyright (C) 2012-2015 Dan Allen, Ryan Waldron and the Asciidoctor Project
Permission is hereby granted, free of charge, to any person obtaining a copy
of this software and associated documentation files (the "Software"), to deal
in the Software without restriction, including without limitation the rights
to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
copies of the Software, and to permit persons to whom the Software is
furnished to do so, subject to the following conditions:
The above copyright notice and this permission notice shall be included in
all copies or substantial portions of the Software.
THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
THE SOFTWARE.
----
This project incorporates portions of the 'Protocol Buffers' project avaialble
under a '3-clause BSD' license.
Copyright 2008, Google Inc.
All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions are
met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above
copyright notice, this list of conditions and the following disclaimer
in the documentation and/or other materials provided with the
distribution.
* Neither the name of Google Inc. nor the names of its
contributors may be used to endorse or promote products derived from
this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
Code generated by the Protocol Buffer compiler is owned by the owner
of the input file used when generating it. This code is not
standalone and requires a support library to be linked with it. This
support library is itself covered by the above license.
----
This project bundles a derivative image for our Orca Logo. This image is
available under the Creative Commons By Attribution 3.0 License.
Creative Commons Legal Code
Attribution 3.0 Unported
CREATIVE COMMONS CORPORATION IS NOT A LAW FIRM AND DOES NOT PROVIDE
LEGAL SERVICES. DISTRIBUTION OF THIS LICENSE DOES NOT CREATE AN
ATTORNEY-CLIENT RELATIONSHIP. CREATIVE COMMONS PROVIDES THIS
INFORMATION ON AN "AS-IS" BASIS. CREATIVE COMMONS MAKES NO WARRANTIES
REGARDING THE INFORMATION PROVIDED, AND DISCLAIMS LIABILITY FOR
DAMAGES RESULTING FROM ITS USE.
License
THE WORK (AS DEFINED BELOW) IS PROVIDED UNDER THE TERMS OF THIS CREATIVE
COMMONS PUBLIC LICENSE ("CCPL" OR "LICENSE"). THE WORK IS PROTECTED BY
COPYRIGHT AND/OR OTHER APPLICABLE LAW. ANY USE OF THE WORK OTHER THAN AS
AUTHORIZED UNDER THIS LICENSE OR COPYRIGHT LAW IS PROHIBITED.
BY EXERCISING ANY RIGHTS TO THE WORK PROVIDED HERE, YOU ACCEPT AND AGREE
TO BE BOUND BY THE TERMS OF THIS LICENSE. TO THE EXTENT THIS LICENSE MAY
BE CONSIDERED TO BE A CONTRACT, THE LICENSOR GRANTS YOU THE RIGHTS
CONTAINED HERE IN CONSIDERATION OF YOUR ACCEPTANCE OF SUCH TERMS AND
CONDITIONS.
1. Definitions
a. "Adaptation" means a work based upon the Work, or upon the Work and
other pre-existing works, such as a translation, adaptation,
derivative work, arrangement of music or other alterations of a
literary or artistic work, or phonogram or performance and includes
cinematographic adaptations or any other form in which the Work may be
recast, transformed, or adapted including in any form recognizably
derived from the original, except that a work that constitutes a
Collection will not be considered an Adaptation for the purpose of
this License. For the avoidance of doubt, where the Work is a musical
work, performance or phonogram, the synchronization of the Work in
timed-relation with a moving image ("synching") will be considered an
Adaptation for the purpose of this License.
b. "Collection" means a collection of literary or artistic works, such as
encyclopedias and anthologies, or performances, phonograms or
broadcasts, or other works or subject matter other than works listed
in Section 1(f) below, which, by reason of the selection and
arrangement of their contents, constitute intellectual creations, in
which the Work is included in its entirety in unmodified form along
with one or more other contributions, each constituting separate and
independent works in themselves, which together are assembled into a
collective whole. A work that constitutes a Collection will not be
considered an Adaptation (as defined above) for the purposes of this
License.
c. "Distribute" means to make available to the public the original and
copies of the Work or Adaptation, as appropriate, through sale or
other transfer of ownership.
d. "Licensor" means the individual, individuals, entity or entities that
offer(s) the Work under the terms of this License.
e. "Original Author" means, in the case of a literary or artistic work,
the individual, individuals, entity or entities who created the Work
or if no individual or entity can be identified, the publisher; and in
addition (i) in the case of a performance the actors, singers,
musicians, dancers, and other persons who act, sing, deliver, declaim,
play in, interpret or otherwise perform literary or artistic works or
expressions of folklore; (ii) in the case of a phonogram the producer
being the person or legal entity who first fixes the sounds of a
performance or other sounds; and, (iii) in the case of broadcasts, the
organization that transmits the broadcast.
f. "Work" means the literary and/or artistic work offered under the terms
of this License including without limitation any production in the
literary, scientific and artistic domain, whatever may be the mode or
form of its expression including digital form, such as a book,
pamphlet and other writing; a lecture, address, sermon or other work
of the same nature; a dramatic or dramatico-musical work; a
choreographic work or entertainment in dumb show; a musical
composition with or without words; a cinematographic work to which are
assimilated works expressed by a process analogous to cinematography;
a work of drawing, painting, architecture, sculpture, engraving or
lithography; a photographic work to which are assimilated works
expressed by a process analogous to photography; a work of applied
art; an illustration, map, plan, sketch or three-dimensional work
relative to geography, topography, architecture or science; a
performance; a broadcast; a phonogram; a compilation of data to the
extent it is protected as a copyrightable work; or a work performed by
a variety or circus performer to the extent it is not otherwise
considered a literary or artistic work.
g. "You" means an individual or entity exercising rights under this
License who has not previously violated the terms of this License with
respect to the Work, or who has received express permission from the
Licensor to exercise rights under this License despite a previous
violation.
h. "Publicly Perform" means to perform public recitations of the Work and
to communicate to the public those public recitations, by any means or
process, including by wire or wireless means or public digital
performances; to make available to the public Works in such a way that
members of the public may access these Works from a place and at a
place individually chosen by them; to perform the Work to the public
by any means or process and the communication to the public of the
performances of the Work, including by public digital performance; to
broadcast and rebroadcast the Work by any means including signs,
sounds or images.
i. "Reproduce" means to make copies of the Work by any means including
without limitation by sound or visual recordings and the right of
fixation and reproducing fixations of the Work, including storage of a
protected performance or phonogram in digital form or other electronic
medium.
2. Fair Dealing Rights. Nothing in this License is intended to reduce,
limit, or restrict any uses free from copyright or rights arising from
limitations or exceptions that are provided for in connection with the
copyright protection under copyright law or other applicable laws.
3. License Grant. Subject to the terms and conditions of this License,
Licensor hereby grants You a worldwide, royalty-free, non-exclusive,
perpetual (for the duration of the applicable copyright) license to
exercise the rights in the Work as stated below:
a. to Reproduce the Work, to incorporate the Work into one or more
Collections, and to Reproduce the Work as incorporated in the
Collections;
b. to create and Reproduce Adaptations provided that any such Adaptation,
including any translation in any medium, takes reasonable steps to
clearly label, demarcate or otherwise identify that changes were made
to the original Work. For example, a translation could be marked "The
original work was translated from English to Spanish," or a
modification could indicate "The original work has been modified.";
c. to Distribute and Publicly Perform the Work including as incorporated
in Collections; and,
d. to Distribute and Publicly Perform Adaptations.
e. For the avoidance of doubt:
i. Non-waivable Compulsory License Schemes. In those jurisdictions in
which the right to collect royalties through any statutory or
compulsory licensing scheme cannot be waived, the Licensor
reserves the exclusive right to collect such royalties for any
exercise by You of the rights granted under this License;
ii. Waivable Compulsory License Schemes. In those jurisdictions in
which the right to collect royalties through any statutory or
compulsory licensing scheme can be waived, the Licensor waives the
exclusive right to collect such royalties for any exercise by You
of the rights granted under this License; and,
iii. Voluntary License Schemes. The Licensor waives the right to
collect royalties, whether individually or, in the event that the
Licensor is a member of a collecting society that administers
voluntary licensing schemes, via that society, from any exercise
by You of the rights granted under this License.
The above rights may be exercised in all media and formats whether now
known or hereafter devised. The above rights include the right to make
such modifications as are technically necessary to exercise the rights in
other media and formats. Subject to Section 8(f), all rights not expressly
granted by Licensor are hereby reserved.
4. Restrictions. The license granted in Section 3 above is expressly made
subject to and limited by the following restrictions:
a. You may Distribute or Publicly Perform the Work only under the terms
of this License. You must include a copy of, or the Uniform Resource
Identifier (URI) for, this License with every copy of the Work You
Distribute or Publicly Perform. You may not offer or impose any terms
on the Work that restrict the terms of this License or the ability of
the recipient of the Work to exercise the rights granted to that
recipient under the terms of the License. You may not sublicense the
Work. You must keep intact all notices that refer to this License and
to the disclaimer of warranties with every copy of the Work You
Distribute or Publicly Perform. When You Distribute or Publicly
Perform the Work, You may not impose any effective technological
measures on the Work that restrict the ability of a recipient of the
Work from You to exercise the rights granted to that recipient under
the terms of the License. This Section 4(a) applies to the Work as
incorporated in a Collection, but this does not require the Collection
apart from the Work itself to be made subject to the terms of this
License. If You create a Collection, upon notice from any Licensor You
must, to the extent practicable, remove from the Collection any credit
as required by Section 4(b), as requested. If You create an
Adaptation, upon notice from any Licensor You must, to the extent
practicable, remove from the Adaptation any credit as required by
Section 4(b), as requested.
b. If You Distribute, or Publicly Perform the Work or any Adaptations or
Collections, You must, unless a request has been made pursuant to
Section 4(a), keep intact all copyright notices for the Work and
provide, reasonable to the medium or means You are utilizing: (i) the
name of the Original Author (or pseudonym, if applicable) if supplied,
and/or if the Original Author and/or Licensor designate another party
or parties (e.g., a sponsor institute, publishing entity, journal) for
attribution ("Attribution Parties") in Licensor's copyright notice,
terms of service or by other reasonable means, the name of such party
or parties; (ii) the title of the Work if supplied; (iii) to the
extent reasonably practicable, the URI, if any, that Licensor
specifies to be associated with the Work, unless such URI does not
refer to the copyright notice or licensing information for the Work;
and (iv) , consistent with Section 3(b), in the case of an Adaptation,
a credit identifying the use of the Work in the Adaptation (e.g.,
"French translation of the Work by Original Author," or "Screenplay
based on original Work by Original Author"). The credit required by
this Section 4 (b) may be implemented in any reasonable manner;
provided, however, that in the case of a Adaptation or Collection, at
a minimum such credit will appear, if a credit for all contributing
authors of the Adaptation or Collection appears, then as part of these
credits and in a manner at least as prominent as the credits for the
other contributing authors. For the avoidance of doubt, You may only
use the credit required by this Section for the purpose of attribution
in the manner set out above and, by exercising Your rights under this
License, You may not implicitly or explicitly assert or imply any
connection with, sponsorship or endorsement by the Original Author,
Licensor and/or Attribution Parties, as appropriate, of You or Your
use of the Work, without the separate, express prior written
permission of the Original Author, Licensor and/or Attribution
Parties.
c. Except as otherwise agreed in writing by the Licensor or as may be
otherwise permitted by applicable law, if You Reproduce, Distribute or
Publicly Perform the Work either by itself or as part of any
Adaptations or Collections, You must not distort, mutilate, modify or
take other derogatory action in relation to the Work which would be
prejudicial to the Original Author's honor or reputation. Licensor
agrees that in those jurisdictions (e.g. Japan), in which any exercise
of the right granted in Section 3(b) of this License (the right to
make Adaptations) would be deemed to be a distortion, mutilation,
modification or other derogatory action prejudicial to the Original
Author's honor and reputation, the Licensor will waive or not assert,
as appropriate, this Section, to the fullest extent permitted by the
applicable national law, to enable You to reasonably exercise Your
right under Section 3(b) of this License (right to make Adaptations)
but not otherwise.
5. Representations, Warranties and Disclaimer
UNLESS OTHERWISE MUTUALLY AGREED TO BY THE PARTIES IN WRITING, LICENSOR
OFFERS THE WORK AS-IS AND MAKES NO REPRESENTATIONS OR WARRANTIES OF ANY
KIND CONCERNING THE WORK, EXPRESS, IMPLIED, STATUTORY OR OTHERWISE,
INCLUDING, WITHOUT LIMITATION, WARRANTIES OF TITLE, MERCHANTIBILITY,
FITNESS FOR A PARTICULAR PURPOSE, NONINFRINGEMENT, OR THE ABSENCE OF
LATENT OR OTHER DEFECTS, ACCURACY, OR THE PRESENCE OF ABSENCE OF ERRORS,
WHETHER OR NOT DISCOVERABLE. SOME JURISDICTIONS DO NOT ALLOW THE EXCLUSION
OF IMPLIED WARRANTIES, SO SUCH EXCLUSION MAY NOT APPLY TO YOU.
6. Limitation on Liability. EXCEPT TO THE EXTENT REQUIRED BY APPLICABLE
LAW, IN NO EVENT WILL LICENSOR BE LIABLE TO YOU ON ANY LEGAL THEORY FOR
ANY SPECIAL, INCIDENTAL, CONSEQUENTIAL, PUNITIVE OR EXEMPLARY DAMAGES
ARISING OUT OF THIS LICENSE OR THE USE OF THE WORK, EVEN IF LICENSOR HAS
BEEN ADVISED OF THE POSSIBILITY OF SUCH DAMAGES.
7. Termination
a. This License and the rights granted hereunder will terminate
automatically upon any breach by You of the terms of this License.
Individuals or entities who have received Adaptations or Collections
from You under this License, however, will not have their licenses
terminated provided such individuals or entities remain in full
compliance with those licenses. Sections 1, 2, 5, 6, 7, and 8 will
survive any termination of this License.
b. Subject to the above terms and conditions, the license granted here is
perpetual (for the duration of the applicable copyright in the Work).
Notwithstanding the above, Licensor reserves the right to release the
Work under different license terms or to stop distributing the Work at
any time; provided, however that any such election will not serve to
withdraw this License (or any other license that has been, or is
required to be, granted under the terms of this License), and this
License will continue in full force and effect unless terminated as
stated above.
8. Miscellaneous
a. Each time You Distribute or Publicly Perform the Work or a Collection,
the Licensor offers to the recipient a license to the Work on the same
terms and conditions as the license granted to You under this License.
b. Each time You Distribute or Publicly Perform an Adaptation, Licensor
offers to the recipient a license to the original Work on the same
terms and conditions as the license granted to You under this License.
c. If any provision of this License is invalid or unenforceable under
applicable law, it shall not affect the validity or enforceability of
the remainder of the terms of this License, and without further action
by the parties to this agreement, such provision shall be reformed to
the minimum extent necessary to make such provision valid and
enforceable.
d. No term or provision of this License shall be deemed waived and no
breach consented to unless such waiver or consent shall be in writing
and signed by the party to be charged with such waiver or consent.
e. This License constitutes the entire agreement between the parties with
respect to the Work licensed here. There are no understandings,
agreements or representations with respect to the Work not specified
here. Licensor shall not be bound by any additional provisions that
may appear in any communication from You. This License may not be
modified without the mutual written agreement of the Licensor and You.
f. The rights granted under, and the subject matter referenced, in this
License were drafted utilizing the terminology of the Berne Convention
for the Protection of Literary and Artistic Works (as amended on
September 28, 1979), the Rome Convention of 1961, the WIPO Copyright
Treaty of 1996, the WIPO Performances and Phonograms Treaty of 1996
and the Universal Copyright Convention (as revised on July 24, 1971).
These rights and subject matter take effect in the relevant
jurisdiction in which the License terms are sought to be enforced
according to the corresponding provisions of the implementation of
those treaty provisions in the applicable national law. If the
standard suite of rights granted under applicable copyright law
includes additional rights not granted under this License, such
additional rights are deemed to be included in the License; this
License is not intended to restrict the license of any rights under
applicable law.
Creative Commons Notice
Creative Commons is not a party to this License, and makes no warranty
whatsoever in connection with the Work. Creative Commons will not be
liable to You or any party on any legal theory for any damages
whatsoever, including without limitation any general, special,
incidental or consequential damages arising in connection to this
license. Notwithstanding the foregoing two (2) sentences, if Creative
Commons has expressly identified itself as the Licensor hereunder, it
shall have all rights and obligations of Licensor.
Except for the limited purpose of indicating to the public that the
Work is licensed under the CCPL, Creative Commons does not authorize
the use by either party of the trademark "Creative Commons" or any
related trademark or logo of Creative Commons without the prior
written consent of Creative Commons. Any permitted use will be in
compliance with Creative Commons' then-current trademark usage
guidelines, as may be published on its website or otherwise made
available upon request from time to time. For the avoidance of doubt,
this trademark restriction does not form part of this License.
Creative Commons may be contacted at https://creativecommons.org/.

View File

@@ -0,0 +1,237 @@
Apache HUDI
Copyright 2019 The Apache Software Foundation
Licensed 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.
This project includes:
aircompressor under Apache License 2.0
An open source Java toolkit for Amazon S3 under Apache License, Version 2.0
Annotation 1.0 under The Apache Software License, Version 2.0
ant under The Apache Software License, Version 2.0
ANTLR 3 Runtime under BSD licence
ANTLR ST4 4.0.4 under BSD licence
AOP alliance under Public Domain
Apache Ant Core under The Apache Software License, Version 2.0
Apache Ant Launcher under The Apache Software License, Version 2.0
Apache Avro under The Apache Software License, Version 2.0
Apache Avro IPC under The Apache Software License, Version 2.0
Apache Avro Mapred API under The Apache Software License, Version 2.0
Apache Calcite Avatica under Apache License, Version 2.0
Apache Calcite Avatica Metrics under Apache License, Version 2.0
Apache Commons Collections under Apache License, Version 2.0
Apache Commons IO under Apache License, Version 2.0
Apache Commons Logging under The Apache Software License, Version 2.0
Apache Curator under The Apache Software License, Version 2.0
Apache Derby Database Engine and Embedded JDBC Driver under Apache 2
Apache Directory API ASN.1 API under The Apache Software License, Version 2.0
Apache Directory LDAP API Utilities under The Apache Software License, Version 2.0
Apache Groovy under The Apache Software License, Version 2.0
Apache Hadoop Annotations under Apache License, Version 2.0
Apache Hadoop Auth under Apache License, Version 2.0
Apache Hadoop Client under Apache License, Version 2.0
Apache Hadoop Common under Apache License, Version 2.0
Apache Hadoop HDFS under Apache License, Version 2.0
Apache HBase - Annotations under Apache License, Version 2.0
Apache HBase - Client under Apache License, Version 2.0
Apache HBase - Protocol under Apache License, Version 2.0
Apache HttpClient under Apache License, Version 2.0
Apache HttpCore under Apache License, Version 2.0
Apache Ivy under The Apache Software License, Version 2.0
Apache Log4j under The Apache Software License, Version 2.0
Apache Log4j 1.x Compatibility API under The Apache Software License, Version 2.0
Apache Log4j API under The Apache Software License, Version 2.0
Apache Log4j Core under The Apache Software License, Version 2.0
Apache Log4j SLF4J Binding under The Apache Software License, Version 2.0
Apache Log4j Web under The Apache Software License, Version 2.0
Apache Parquet Avro under The Apache Software License, Version 2.0
Apache Parquet Avro (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Column under The Apache Software License, Version 2.0
Apache Parquet Column (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Common under The Apache Software License, Version 2.0
Apache Parquet Common (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Encodings under The Apache Software License, Version 2.0
Apache Parquet Encodings (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Format (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Generator (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Hadoop under The Apache Software License, Version 2.0
Apache Parquet Hadoop (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Hadoop Bundle under The Apache Software License, Version 2.0
Apache Parquet Hadoop Bundle (Incubating) under The Apache Software License, Version 2.0
Apache Parquet Jackson under The Apache Software License, Version 2.0
Apache Parquet Jackson (Incubating) under The Apache Software License, Version 2.0
Apache Thrift under The Apache Software License, Version 2.0
Apache Twill API under The Apache Software License, Version 2.0
Apache Twill common library under The Apache Software License, Version 2.0
Apache Twill core library under The Apache Software License, Version 2.0
Apache Twill discovery service API under The Apache Software License, Version 2.0
Apache Twill discovery service implementations under The Apache Software License, Version 2.0
Apache Twill ZooKeeper client library under The Apache Software License, Version 2.0
Apache Velocity under The Apache Software License, Version 2.0
ApacheDS I18n under The Apache Software License, Version 2.0
ApacheDS Protocol Kerberos Codec under The Apache Software License, Version 2.0
ASM Commons under 3-Clause BSD License
ASM Core under 3-Clause BSD License
ASM Tree under 3-Clause BSD License
BoneCP :: Core Library under Apache v2
Calcite Core under Apache License, Version 2.0
Calcite Druid under Apache License, Version 2.0
Calcite Linq4j under Apache License, Version 2.0
com.twitter.common:objectsize under Apache License, Version 2.0
Commons BeanUtils Core under The Apache Software License, Version 2.0
Commons CLI under The Apache Software License, Version 2.0
Commons Codec under The Apache Software License, Version 2.0
Commons Compiler under New BSD License
Commons Compress under The Apache Software License, Version 2.0
Commons Configuration under The Apache Software License, Version 2.0
Commons Daemon under The Apache Software License, Version 2.0
Commons DBCP under The Apache Software License, Version 2.0
Commons Lang under The Apache Software License, Version 2.0
Commons Math under The Apache Software License, Version 2.0
Commons Net under The Apache Software License, Version 2.0
Commons Pool under The Apache Software License, Version 2.0
commons-beanutils under Apache License
Curator Client under The Apache Software License, Version 2.0
Curator Framework under The Apache Software License, Version 2.0
Curator Recipes under The Apache Software License, Version 2.0
Data Mapper for Jackson under The Apache Software License, Version 2.0
DataNucleus Core under The Apache Software License, Version 2.0
DataNucleus JDO API plugin under The Apache Software License, Version 2.0
DataNucleus RDBMS plugin under The Apache Software License, Version 2.0
Digester under The Apache Software License, Version 2.0
Disruptor Framework under The Apache Software License, Version 2.0
eigenbase-properties under Apache License, Version 2.0
EL under The Apache Software License, Version 2.0
fastutil under Apache License, Version 2.0
Findbugs Annotations under Apache License under Apache License, Version 2.0
FindBugs-jsr305 under The Apache Software License, Version 2.0
Fluent API for Apache HttpClient under Apache License, Version 2.0
Glassfish Jasper under CDDL 1.0
Glassfish Jasper API under Apache License Version 2.0
Google Guice - Core Library under The Apache Software License, Version 2.0
Google Guice - Extensions - AssistedInject under The Apache Software License, Version 2.0
Google Guice - Extensions - Servlet under The Apache Software License, Version 2.0
Gson under The Apache Software License, Version 2.0
Guava: Google Core Libraries for Java under The Apache Software License, Version 2.0
Hadoop Metrics2 Reporter for Dropwizard Metrics under Apache License, Version 2.0
hadoop-mapreduce-client-app under Apache License, Version 2.0
hadoop-mapreduce-client-common under Apache License, Version 2.0
hadoop-mapreduce-client-core under Apache License, Version 2.0
hadoop-mapreduce-client-jobclient under Apache License, Version 2.0
hadoop-mapreduce-client-shuffle under Apache License, Version 2.0
hadoop-yarn-api under Apache License, Version 2.0
hadoop-yarn-client under Apache License, Version 2.0
hadoop-yarn-common under Apache License, Version 2.0
hadoop-yarn-registry under Apache License, Version 2.0
hadoop-yarn-server-applicationhistoryservice under Apache License, Version 2.0
hadoop-yarn-server-common under Apache License, Version 2.0
hadoop-yarn-server-resourcemanager under Apache License, Version 2.0
hadoop-yarn-server-web-proxy under Apache License, Version 2.0
Hamcrest Core under New BSD License
HBase - Common under The Apache Software License, Version 2.0
HBase - Hadoop Compatibility under The Apache Software License, Version 2.0
HBase - Hadoop Two Compatibility under The Apache Software License, Version 2.0
HBase - Prefix Tree under The Apache Software License, Version 2.0
HBase - Procedure under The Apache Software License, Version 2.0
HBase - Server under The Apache Software License, Version 2.0
HikariCP under The Apache Software License, Version 2.0
Hive Common under The Apache Software License, Version 2.0
Hive JDBC under The Apache Software License, Version 2.0
Hive Llap Client under The Apache Software License, Version 2.0
Hive Llap Common under The Apache Software License, Version 2.0
Hive Llap Server under The Apache Software License, Version 2.0
Hive Llap Tez under The Apache Software License, Version 2.0
Hive Metastore under The Apache Software License, Version 2.0
Hive Query Language under The Apache Software License, Version 2.0
Hive Serde under The Apache Software License, Version 2.0
Hive Service under The Apache Software License, Version 2.0
Hive Service RPC under The Apache Software License, Version 2.0
Hive Shims under The Apache Software License, Version 2.0
Hive Shims 0.23 under The Apache Software License, Version 2.0
Hive Shims Common under The Apache Software License, Version 2.0
Hive Shims Scheduler under The Apache Software License, Version 2.0
Hive Storage API under Apache License, Version 2.0
Hive Vector-Code-Gen Utilities under The Apache Software License, Version 2.0
hoodie-common under Apache License, Version 2.0
hoodie-hadoop-mr under Apache License, Version 2.0
hoodie-hive under Apache License, Version 2.0
htrace-core under The Apache Software License, Version 2.0
HttpClient under Apache License
Jackson under The Apache Software License, Version 2.0
Jackson Integration for Metrics under Apache License 2.0
Jackson-annotations under The Apache Software License, Version 2.0
Jackson-core under The Apache Software License, Version 2.0
jackson-databind under The Apache Software License, Version 2.0
jamon-runtime under Mozilla Public License Version 1.1
Janino under New BSD License
jasper-compiler under The Apache Software License, Version 2.0
jasper-runtime under The Apache Software License, Version 2.0
Java Authentication SPI for Containers under The Apache Software License, Version 2.0
java-xmlbuilder under Apache License, Version 2.0
JavaBeans Activation Framework (JAF) under Common Development and Distribution License (CDDL) v1.0
JavaMail API under Common Development and Distribution License (CDDL) v1.0
javax.inject under The Apache Software License, Version 2.0
Javolution under BSD License
JAX-RS provider for JSON content type under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
JAXB RI under CDDL 1.1 or GPL2 w/ CPE
JCodings under MIT License
jcommander under Apache 2.0
JDO API under Apache 2
jersey-client under CDDL 1.1 or GPL2 w/ CPE
jersey-core under CDDL 1.1 or GPL2 w/ CPE
jersey-guice under CDDL 1.1 or GPL2 w/ CPE
jersey-json under CDDL 1.1 or GPL2 w/ CPE
jersey-server under CDDL 1.1 or GPL2 w/ CPE
Jettison under Apache License, Version 2.0
Jetty :: Aggregate :: All core Jetty under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
Jetty Orbit :: Servlet API under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
Jetty Server under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
Jetty SSLEngine under Apache License Version 2
Jetty Utilities under Apache Software License - Version 2.0 or Eclipse Public License - Version 1.0
Joda-Time under Apache 2
Joni under MIT License
JPam under The Apache Software License, Version 2.0
JSch under BSD
jsp-api under CDDL
JTA 1.1 under The Apache Software License, Version 2.0
JUnit under Common Public License Version 1.0
JVM Integration for Metrics under Apache License 2.0
Kryo under New BSD License
Kryo Shaded under 3-Clause BSD License
leveldbjni-all under The BSD 3-Clause License
Metrics Core under Apache License 2.0
Metrics Core Library under Apache License 2.0
MinLog under New BSD License
Mockito under The MIT License
Netty/All-in-One under Apache License, Version 2.0
Objenesis under Apache 2
Open JSON under The Apache Software License, Version 2.0
opencsv under Apache 2
ORC Core under Apache License, Version 2.0
org.pentaho:pentaho-aggdesigner-algorithm under Apache License, Version 2.0
ParaNamer Core under BSD
Protocol Buffer Java API under New BSD license
ReflectASM under New BSD License
RocksDB JNI under Apache License 2.0 or GNU General Public License, version 2
Servlet Specification 2.5 API under CDDL 1.0
Servlet Specification API under Apache License Version 2.0
servlet-api under CDDL
SLF4J API Module under MIT License
SLF4J LOG4J-12 Binding under MIT License
Slider Core under Apache License, Version 2.0
Snappy for Java under The Apache Software License, Version 2.0
StAX API under The Apache Software License, Version 2.0
Tephra API under The Apache Software License, Version 2.0
Tephra Core under The Apache Software License, Version 2.0
Tephra HBase 1.0 Compatibility under The Apache Software License, Version 2.0
The Netty Project under Apache License, Version 2.0
Xerces2 Java Parser under The Apache Software License, Version 2.0
XML Commons External Components XML APIs under The Apache Software License, Version 2.0
Xml Compatibility extensions for Jackson under The Apache Software License, Version 2.0 or GNU Lesser General Public License (LGPL), Version 2.1
xmlenc Library under The BSD License
XZ for Java under Public Domain
zookeeper under Apache License, Version 2.0

View File

@@ -0,0 +1,377 @@
/*
* 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 org.apache.hudi.hive;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import com.google.common.collect.Lists;
import java.io.IOException;
import java.net.URISyntaxException;
import java.util.List;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent;
import org.apache.hudi.hive.HoodieHiveClient.PartitionEvent.PartitionEventType;
import org.apache.hudi.hive.util.SchemaUtil;
import org.joda.time.DateTime;
import org.junit.Before;
import org.junit.Test;
import parquet.schema.MessageType;
import parquet.schema.OriginalType;
import parquet.schema.PrimitiveType;
@SuppressWarnings("ConstantConditions")
public class HiveSyncToolTest {
@Before
public void setUp() throws IOException, InterruptedException, URISyntaxException {
TestUtil.setUp();
}
@Before
public void teardown() throws IOException, InterruptedException {
TestUtil.clear();
}
/**
* Testing converting array types to Hive field declaration strings, according to the Parquet-113
* spec: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists
*/
@Test
public void testSchemaConvertArray() throws IOException {
// Testing the 3-level annotation structure
MessageType schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup()
.optional(PrimitiveType.PrimitiveTypeName.INT32).named("element")
.named("list").named("int_list").named("ArrayOfInts");
String schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`int_list` ARRAY< int>", schemaString);
// A array of arrays
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup().requiredGroup()
.as(OriginalType.LIST).repeatedGroup()
.required(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("list")
.named("element").named("list").named("int_list_list").named("ArrayOfArrayOfInts");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`int_list_list` ARRAY< ARRAY< int>>", schemaString);
// A list of integers
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST)
.repeated(PrimitiveType.PrimitiveTypeName.INT32).named("element").named("int_list")
.named("ArrayOfInts");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`int_list` ARRAY< int>", schemaString);
// A list of structs with two fields
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup()
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str")
.required(PrimitiveType.PrimitiveTypeName.INT32).named("num").named("element")
.named("tuple_list").named("ArrayOfTuples");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`tuple_list` ARRAY< STRUCT< `str` : binary, `num` : int>>", schemaString);
// A list of structs with a single field
// For this case, since the inner group name is "array", we treat the
// element type as a one-element struct.
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup()
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str").named("array")
.named("one_tuple_list").named("ArrayOfOneTuples");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
// A list of structs with a single field
// For this case, since the inner group name ends with "_tuple", we also treat the
// element type as a one-element struct.
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup()
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str")
.named("one_tuple_list_tuple").named("one_tuple_list").named("ArrayOfOneTuples2");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< STRUCT< `str` : binary>>", schemaString);
// A list of structs with a single field
// Unlike the above two cases, for this the element type is the type of the
// only field in the struct.
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup()
.required(PrimitiveType.PrimitiveTypeName.BINARY).named("str")
.named("one_tuple_list").named("one_tuple_list").named("ArrayOfOneTuples3");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`one_tuple_list` ARRAY< binary>", schemaString);
// A list of maps
schema = parquet.schema.Types.buildMessage().optionalGroup()
.as(parquet.schema.OriginalType.LIST).repeatedGroup().as(OriginalType.MAP)
.repeatedGroup().as(OriginalType.MAP_KEY_VALUE)
.required(PrimitiveType.PrimitiveTypeName.BINARY).as(OriginalType.UTF8)
.named("string_key").required(PrimitiveType.PrimitiveTypeName.INT32)
.named("int_value").named("key_value").named("array").named("map_list")
.named("ArrayOfMaps");
schemaString = SchemaUtil.generateSchemaString(schema);
assertEquals("`map_list` ARRAY< MAP< string, int>>", schemaString);
}
@Test
public void testBasicSync() throws Exception {
String commitTime = "100";
TestUtil.createCOWDataset(commitTime, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
hiveClient.doesTableExist());
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
tool.syncHoodieTable();
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes",
hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field",
hiveClient.getTableSchema().size(), hiveClient.getDataSchema().getColumns().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
commitTime, hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncIncremental() throws Exception {
String commitTime1 = "100";
TestUtil.createCOWDataset(commitTime1, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
tool.syncHoodieTable();
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
commitTime1, hiveClient.getLastCommitTimeSynced().get());
// Now lets create more parititions and these are the only ones which needs to be synced
DateTime dateTime = DateTime.now().plusDays(6);
String commitTime2 = "101";
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
// Lets do the sync
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(
Option.of(commitTime1));
assertEquals("We should have one partition written after 100 commit", 1,
writtenPartitionsSince.size());
List<Partition> hivePartitions = hiveClient.scanTablePartitions();
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions,
writtenPartitionsSince);
assertEquals("There should be only one paritition event", 1, partitionEvents.size());
assertEquals("The one partition event must of type ADD", PartitionEventType.ADD,
partitionEvents.iterator().next().eventType);
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
// Sync should add the one partition
assertEquals("The one partition we wrote should be added to hive", 6,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 101", commitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncIncrementalWithSchemaEvolution() throws Exception {
String commitTime1 = "100";
TestUtil.createCOWDataset(commitTime1, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
tool.syncHoodieTable();
int fields = hiveClient.getTableSchema().size();
// Now lets create more parititions and these are the only ones which needs to be synced
DateTime dateTime = DateTime.now().plusDays(6);
String commitTime2 = "101";
TestUtil.addCOWPartitions(1, false, dateTime, commitTime2);
// Lets do the sync
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
assertEquals("Hive Schema has evolved and should not be 3 more field", fields + 3,
hiveClient.getTableSchema().size());
assertEquals("Hive Schema has evolved - Field favorite_number has evolved from int to long",
"BIGINT", hiveClient.getTableSchema().get("favorite_number"));
assertTrue("Hive Schema has evolved - Field favorite_movie was added",
hiveClient.getTableSchema().containsKey("favorite_movie"));
// Sync should add the one partition
assertEquals("The one partition we wrote should be added to hive", 6,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 101", commitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncMergeOnRead() throws Exception {
String commitTime = "100";
String deltaCommitTime = "101";
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + " should not exist initially",
hiveClient.doesTableExist());
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
tool.syncHoodieTable();
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + " should exist after sync completes",
hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field",
hiveClient.getTableSchema().size(),
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
deltaCommitTime, hiveClient.getLastCommitTimeSynced().get());
// Now lets create more parititions and these are the only ones which needs to be synced
DateTime dateTime = DateTime.now().plusDays(6);
String commitTime2 = "102";
String deltaCommitTime2 = "103";
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2);
// Lets do the sync
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
hiveClient.getTableSchema().size(),
SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
// Sync should add the one partition
assertEquals("The 2 partitions we wrote should be added to hive", 6,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
hiveClient.getLastCommitTimeSynced().get());
}
@Test
public void testSyncMergeOnReadRT()
throws Exception {
String commitTime = "100";
String deltaCommitTime = "101";
String roTablename = TestUtil.hiveSyncConfig.tableName;
TestUtil.hiveSyncConfig.tableName =
TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE;
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
HoodieHiveClient hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE
+ " should not exist initially", hiveClientRT.doesTableExist());
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
tool.syncHoodieTable();
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE
+ " should exist after sync completes", hiveClientRT.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition field",
hiveClientRT.getTableSchema().size(),
SchemaTestUtil.getSimpleSchema().getFields().size() + 1);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClientRT.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
deltaCommitTime, hiveClientRT.getLastCommitTimeSynced().get());
// Now lets create more parititions and these are the only ones which needs to be synced
DateTime dateTime = DateTime.now().plusDays(6);
String commitTime2 = "102";
String deltaCommitTime2 = "103";
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
TestUtil.addMORPartitions(1, true, false, dateTime, commitTime2, deltaCommitTime2);
// Lets do the sync
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
TestUtil.fileSystem);
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
hiveClientRT.getTableSchema().size(),
SchemaTestUtil.getEvolvedSchema().getFields().size() + 1);
// Sync should add the one partition
assertEquals("The 2 partitions we wrote should be added to hive", 6,
hiveClientRT.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
hiveClientRT.getLastCommitTimeSynced().get());
TestUtil.hiveSyncConfig.tableName = roTablename;
}
@Test
public void testMultiPartitionKeySync()
throws Exception {
String commitTime = "100";
TestUtil.createCOWDataset(commitTime, 5);
HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(TestUtil.hiveSyncConfig);
hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName();
hiveSyncConfig.tableName = "multi_part_key";
hiveSyncConfig.partitionFields = Lists.newArrayList("year", "month", "day");
TestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig,
TestUtil.getHiveConf(), TestUtil.fileSystem);
assertFalse("Table " + hiveSyncConfig.tableName + " should not exist initially",
hiveClient.doesTableExist());
// Lets do the sync
HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
tool.syncHoodieTable();
assertTrue("Table " + hiveSyncConfig.tableName + " should exist after sync completes",
hiveClient.doesTableExist());
assertEquals("Hive Schema should match the dataset schema + partition fields",
hiveClient.getTableSchema().size(), hiveClient.getDataSchema().getColumns().size() + 3);
assertEquals("Table partitions should match the number of partitions we wrote", 5,
hiveClient.scanTablePartitions().size());
assertEquals("The last commit that was sycned should be updated in the TBLPROPERTIES",
commitTime, hiveClient.getLastCommitTimeSynced().get());
}
}

View File

@@ -0,0 +1,368 @@
/*
* 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 org.apache.hudi.hive;
import static org.junit.Assert.fail;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import java.io.File;
import java.io.IOException;
import java.net.URISyntaxException;
import java.nio.charset.StandardCharsets;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.MiniDFSCluster;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hive.service.server.HiveServer2;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.BloomFilter;
import org.apache.hudi.common.minicluster.HdfsTestService;
import org.apache.hudi.common.minicluster.ZookeeperTestService;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieDataFile;
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTimeline;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.util.FSUtils;
import org.apache.hudi.common.util.SchemaTestUtil;
import org.apache.hudi.hive.util.HiveTestService;
import org.apache.parquet.avro.AvroSchemaConverter;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.zookeeper.server.ZooKeeperServer;
import org.joda.time.DateTime;
import org.joda.time.format.DateTimeFormat;
import org.joda.time.format.DateTimeFormatter;
import org.junit.runners.model.InitializationError;
@SuppressWarnings("SameParameterValue")
public class TestUtil {
private static MiniDFSCluster dfsCluster;
private static ZooKeeperServer zkServer;
private static HiveServer2 hiveServer;
private static Configuration configuration;
static HiveSyncConfig hiveSyncConfig;
private static DateTimeFormatter dtfOut;
static FileSystem fileSystem;
private static Set<String> createdTablesSet = Sets.newHashSet();
public static void setUp() throws IOException, InterruptedException, URISyntaxException {
if (dfsCluster == null) {
HdfsTestService service = new HdfsTestService();
dfsCluster = service.start(true);
configuration = service.getHadoopConf();
}
if (zkServer == null) {
ZookeeperTestService zkService = new ZookeeperTestService(configuration);
zkServer = zkService.start();
}
if (hiveServer == null) {
HiveTestService hiveService = new HiveTestService(configuration);
hiveServer = hiveService.start();
}
fileSystem = FileSystem.get(configuration);
hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.jdbcUrl = "jdbc:hive2://127.0.0.1:9999/";
hiveSyncConfig.databaseName = "hdrone_test";
hiveSyncConfig.hiveUser = "";
hiveSyncConfig.hivePass = "";
hiveSyncConfig.databaseName = "testdb";
hiveSyncConfig.tableName = "test1";
hiveSyncConfig.basePath = "/tmp/hdfs/HiveSyncToolTest/";
hiveSyncConfig.assumeDatePartitioning = true;
hiveSyncConfig.usePreApacheInputFormat = false;
hiveSyncConfig.partitionFields = Lists.newArrayList("datestr");
dtfOut = DateTimeFormat.forPattern("yyyy/MM/dd");
clear();
}
static void clear() throws IOException {
fileSystem.delete(new Path(hiveSyncConfig.basePath), true);
HoodieTableMetaClient
.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
HoodieHiveClient client = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(),
fileSystem);
for (String tableName : createdTablesSet) {
client.updateHiveSQL("drop table if exists " + tableName);
}
createdTablesSet.clear();
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);
}
static HiveConf getHiveConf() {
return hiveServer.getHiveConf();
}
@SuppressWarnings("unused")
public static void shutdown() {
if (hiveServer != null) {
hiveServer.stop();
}
if (dfsCluster != null) {
dfsCluster.shutdown();
}
if (zkServer != null) {
zkServer.shutdown();
}
}
static void createCOWDataset(String commitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath);
FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient
.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.COPY_ON_WRITE,
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
boolean result = fileSystem.mkdirs(path);
checkResult(result);
DateTime dateTime = DateTime.now();
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime,
commitTime);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
createCommitFile(commitMetadata, commitTime);
}
static void createMORDataset(String commitTime, String deltaCommitTime, int numberOfPartitions)
throws IOException, InitializationError, URISyntaxException, InterruptedException {
Path path = new Path(hiveSyncConfig.basePath);
FileUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
HoodieTableMetaClient
.initTableType(configuration, hiveSyncConfig.basePath, HoodieTableType.MERGE_ON_READ,
hiveSyncConfig.tableName, HoodieAvroPayload.class.getName());
boolean result = fileSystem.mkdirs(path);
checkResult(result);
DateTime dateTime = DateTime.now();
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, dateTime,
commitTime);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName
+ HiveSyncTool.SUFFIX_REALTIME_TABLE);
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
commitMetadata.getPartitionToWriteStats().forEach(
(key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l)));
createCompactionCommitFile(compactionMetadata, commitTime);
// Write a delta commit
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(),
true);
createDeltaCommitFile(deltaMetadata, deltaCommitTime);
}
static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple,
DateTime startFrom, String commitTime)
throws IOException, URISyntaxException, InterruptedException {
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions,
isParquetSchemaSimple, startFrom, commitTime);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
createCommitFile(commitMetadata, commitTime);
}
static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple,
boolean isLogSchemaSimple, DateTime startFrom, String commitTime, String deltaCommitTime)
throws IOException, URISyntaxException, InterruptedException {
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions,
isParquetSchemaSimple, startFrom, commitTime);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName
+ HiveSyncTool.SUFFIX_REALTIME_TABLE);
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
commitMetadata.getPartitionToWriteStats().forEach(
(key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l)));
createCompactionCommitFile(compactionMetadata, commitTime);
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(),
isLogSchemaSimple);
createDeltaCommitFile(deltaMetadata, deltaCommitTime);
}
private static HoodieCommitMetadata createLogFiles(
Map<String, List<HoodieWriteStat>> partitionWriteStats, boolean isLogSchemaSimple)
throws InterruptedException, IOException, URISyntaxException {
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
for (Entry<String, List<HoodieWriteStat>> wEntry : partitionWriteStats.entrySet()) {
String partitionPath = wEntry.getKey();
for (HoodieWriteStat wStat : wEntry.getValue()) {
Path path = new Path(wStat.getPath());
HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(path));
HoodieLogFile logFile = generateLogData(path, isLogSchemaSimple);
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
writeStat.setFileId(dataFile.getFileId());
writeStat.setPath(logFile.getPath().toString());
commitMetadata.addWriteStat(partitionPath, writeStat);
}
}
return commitMetadata;
}
private static HoodieCommitMetadata createPartitions(int numberOfPartitions,
boolean isParquetSchemaSimple, DateTime startFrom, String commitTime)
throws IOException, URISyntaxException, InterruptedException {
startFrom = startFrom.withTimeAtStartOfDay();
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
for (int i = 0; i < numberOfPartitions; i++) {
String partitionPath = dtfOut.print(startFrom);
Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath);
fileSystem.makeQualified(partPath);
fileSystem.mkdirs(partPath);
List<HoodieWriteStat> writeStats = createTestData(partPath, isParquetSchemaSimple,
commitTime);
startFrom = startFrom.minusDays(1);
writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s));
}
return commitMetadata;
}
private static List<HoodieWriteStat> createTestData(Path partPath, boolean isParquetSchemaSimple,
String commitTime) throws IOException, URISyntaxException, InterruptedException {
List<HoodieWriteStat> writeStats = Lists.newArrayList();
for (int i = 0; i < 5; i++) {
// Create 5 files
String fileId = UUID.randomUUID().toString();
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(commitTime,
"1-0-1", fileId));
generateParquetData(filePath, isParquetSchemaSimple);
HoodieWriteStat writeStat = new HoodieWriteStat();
writeStat.setFileId(fileId);
writeStat.setPath(filePath.toString());
writeStats.add(writeStat);
}
return writeStats;
}
@SuppressWarnings({"unchecked", "deprecation"})
private static void generateParquetData(Path filePath, boolean isParquetSchemaSimple)
throws IOException, URISyntaxException, InterruptedException {
Schema schema = (isParquetSchemaSimple ? SchemaTestUtil.getSimpleSchema()
: SchemaTestUtil.getEvolvedSchema());
org.apache.parquet.schema.MessageType parquetSchema = new AvroSchemaConverter().convert(schema);
BloomFilter filter = new BloomFilter(1000, 0.0001);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(parquetSchema, schema, filter);
ParquetWriter writer = new ParquetWriter(filePath, writeSupport, CompressionCodecName.GZIP,
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE,
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
ParquetWriter.DEFAULT_WRITER_VERSION, fileSystem.getConf());
List<IndexedRecord> testRecords = (isParquetSchemaSimple ? SchemaTestUtil
.generateTestRecords(0, 100)
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
testRecords.forEach(s -> {
try {
writer.write(s);
} catch (IOException e) {
fail("IOException while writing test records as parquet" + e.toString());
}
});
writer.close();
}
private static HoodieLogFile generateLogData(Path parquetFilePath, boolean isLogSchemaSimple)
throws IOException, InterruptedException, URISyntaxException {
Schema schema = (isLogSchemaSimple ? SchemaTestUtil.getSimpleSchema()
: SchemaTestUtil.getEvolvedSchema());
HoodieDataFile dataFile = new HoodieDataFile(fileSystem.getFileStatus(parquetFilePath));
// Write a log file for this parquet file
Writer logWriter = HoodieLogFormat.newWriterBuilder().onParentPath(parquetFilePath.getParent())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
.withFileId(dataFile.getFileId())
.overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build();
List<IndexedRecord> records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100)
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
Map<HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime());
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
HoodieAvroDataBlock dataBlock = new HoodieAvroDataBlock(records, header);
logWriter.appendBlock(dataBlock);
logWriter.close();
return logWriter.getLogFile();
}
private static void checkResult(boolean result) throws InitializationError {
if (!result) {
throw new InitializationError("Could not initialize");
}
}
private static void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime)
throws IOException {
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
Path fullPath = new Path(
hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(
commitTime));
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
fsout.write(bytes);
fsout.close();
}
private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata,
String commitTime) throws IOException {
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
Path fullPath = new Path(
hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeCommitFileName(
commitTime));
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
fsout.write(bytes);
fsout.close();
}
private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata,
String deltaCommitTime) throws IOException {
byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
Path fullPath = new Path(
hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline
.makeDeltaFileName(
deltaCommitTime));
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
fsout.write(bytes);
fsout.close();
}
public static Set<String> getCreatedTablesSet() {
return createdTablesSet;
}
}

View File

@@ -0,0 +1,326 @@
/*
* 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 org.apache.hudi.hive.util;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.net.SocketException;
import java.util.Map;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.commons.io.FileUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.hadoop.hive.metastore.HiveMetaStore;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.IHMSHandler;
import org.apache.hadoop.hive.metastore.TSetIpAddressProcessor;
import org.apache.hadoop.hive.metastore.TUGIBasedProcessor;
import org.apache.hadoop.hive.metastore.api.MetaException;
import org.apache.hadoop.hive.thrift.TUGIContainingTransport;
import org.apache.hive.service.server.HiveServer2;
import org.apache.hudi.common.model.HoodieTestUtils;
import org.apache.thrift.TProcessor;
import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.server.TServer;
import org.apache.thrift.server.TThreadPoolServer;
import org.apache.thrift.transport.TFramedTransport;
import org.apache.thrift.transport.TServerSocket;
import org.apache.thrift.transport.TServerTransport;
import org.apache.thrift.transport.TSocket;
import org.apache.thrift.transport.TTransport;
import org.apache.thrift.transport.TTransportException;
import org.apache.thrift.transport.TTransportFactory;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class HiveTestService {
private static final Logger LOG = LoggerFactory.getLogger(HiveTestService.class);
private static final int CONNECTION_TIMEOUT = 30000;
/**
* Configuration settings
*/
private Configuration hadoopConf;
private String workDir;
private String bindIP = "127.0.0.1";
private int metastorePort = 9083;
private int serverPort = 9999;
private boolean clean = true;
private Map<String, String> sysProps = Maps.newHashMap();
private ExecutorService executorService;
private TServer tServer;
private HiveServer2 hiveServer;
public HiveTestService(Configuration configuration) {
this.workDir = Files.createTempDir().getAbsolutePath();
}
public Configuration getHadoopConf() {
return hadoopConf;
}
public HiveServer2 start() throws IOException {
Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster.");
if (hadoopConf == null) {
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
}
String localHiveLocation = getHiveLocation(workDir);
if (clean) {
LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh.");
File file = new File(localHiveLocation);
FileUtils.deleteDirectory(file);
}
HiveConf serverConf = configureHive(hadoopConf, localHiveLocation);
executorService = Executors.newSingleThreadExecutor();
tServer = startMetaStore(bindIP, metastorePort, serverConf);
hiveServer = startHiveServer(serverConf);
String serverHostname;
if (bindIP.equals("0.0.0.0")) {
serverHostname = "localhost";
} else {
serverHostname = bindIP;
}
if (!waitForServerUp(serverConf, serverHostname, metastorePort, CONNECTION_TIMEOUT)) {
throw new IOException("Waiting for startup of standalone server");
}
LOG.info("Hive Minicluster service started.");
return hiveServer;
}
public void stop() throws IOException {
resetSystemProperties();
if (tServer != null) {
tServer.stop();
}
if (hiveServer != null) {
hiveServer.stop();
}
LOG.info("Hive Minicluster service shut down.");
tServer = null;
hiveServer = null;
hadoopConf = null;
}
private HiveConf configureHive(Configuration conf, String localHiveLocation) throws IOException {
conf.set("hive.metastore.local", "false");
conf.set(HiveConf.ConfVars.METASTOREURIS.varname, "thrift://" + bindIP + ":" + metastorePort);
conf.set(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST.varname, bindIP);
conf.setInt(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, serverPort);
// The following line to turn of SASL has no effect since HiveAuthFactory calls
// 'new HiveConf()'. This is fixed by https://issues.apache.org/jira/browse/HIVE-6657,
// in Hive 0.14.
// As a workaround, the property is set in hive-site.xml in this module.
//conf.set(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION.varname, "NOSASL");
File localHiveDir = new File(localHiveLocation);
localHiveDir.mkdirs();
File metastoreDbDir = new File(localHiveDir, "metastore_db");
conf.set(HiveConf.ConfVars.METASTORECONNECTURLKEY.varname,
"jdbc:derby:" + metastoreDbDir.getPath() + ";create=true");
File derbyLogFile = new File(localHiveDir, "derby.log");
derbyLogFile.createNewFile();
setSystemProperty("derby.stream.error.file", derbyLogFile.getPath());
conf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, Files.createTempDir().getAbsolutePath());
conf.set("datanucleus.schema.autoCreateTables", "true");
conf.set("hive.metastore.schema.verification", "false");
setSystemProperty("derby.stream.error.file", derbyLogFile.getPath());
return new HiveConf(conf, this.getClass());
}
private boolean waitForServerUp(HiveConf serverConf, String hostname, int port, int timeout) {
long start = System.currentTimeMillis();
while (true) {
try {
new HiveMetaStoreClient(serverConf);
return true;
} catch (MetaException e) {
// ignore as this is expected
LOG.info("server " + hostname + ":" + port + " not up " + e);
}
if (System.currentTimeMillis() > start + timeout) {
break;
}
try {
Thread.sleep(250);
} catch (InterruptedException e) {
// ignore
}
}
return false;
}
private void setSystemProperty(String name, String value) {
if (!sysProps.containsKey(name)) {
String currentValue = System.getProperty(name);
sysProps.put(name, currentValue);
}
if (value != null) {
System.setProperty(name, value);
} else {
System.getProperties().remove(name);
}
}
private void resetSystemProperties() {
for (Map.Entry<String, String> entry : sysProps.entrySet()) {
if (entry.getValue() != null) {
System.setProperty(entry.getKey(), entry.getValue());
} else {
System.getProperties().remove(entry.getKey());
}
}
sysProps.clear();
}
private static String getHiveLocation(String baseLocation) {
return baseLocation + Path.SEPARATOR + "hive";
}
private HiveServer2 startHiveServer(HiveConf serverConf) {
HiveServer2 hiveServer = new HiveServer2();
hiveServer.init(serverConf);
hiveServer.start();
return hiveServer;
}
// XXX: From org.apache.hadoop.hive.metastore.HiveMetaStore,
// with changes to support binding to a specified IP address (not only 0.0.0.0)
private static final class ChainedTTransportFactory extends TTransportFactory {
private final TTransportFactory parentTransFactory;
private final TTransportFactory childTransFactory;
private ChainedTTransportFactory(TTransportFactory parentTransFactory,
TTransportFactory childTransFactory) {
this.parentTransFactory = parentTransFactory;
this.childTransFactory = childTransFactory;
}
@Override
public TTransport getTransport(TTransport trans) {
return childTransFactory.getTransport(parentTransFactory.getTransport(trans));
}
}
private static final class TServerSocketKeepAlive extends TServerSocket {
public TServerSocketKeepAlive(int port) throws TTransportException {
super(port, 0);
}
public TServerSocketKeepAlive(InetSocketAddress address) throws TTransportException {
super(address, 0);
}
@Override
protected TSocket acceptImpl() throws TTransportException {
TSocket ts = super.acceptImpl();
try {
ts.getSocket().setKeepAlive(true);
} catch (SocketException e) {
throw new TTransportException(e);
}
return ts;
}
}
public TServer startMetaStore(String forceBindIP, int port, HiveConf conf) throws IOException {
try {
// Server will create new threads up to max as necessary. After an idle
// period, it will destory threads to keep the number of threads in the
// pool to min.
int minWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMINTHREADS);
int maxWorkerThreads = conf.getIntVar(HiveConf.ConfVars.METASTORESERVERMAXTHREADS);
boolean tcpKeepAlive = conf.getBoolVar(HiveConf.ConfVars.METASTORE_TCP_KEEP_ALIVE);
boolean useFramedTransport = conf.getBoolVar(
HiveConf.ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT);
// don't support SASL yet
//boolean useSasl = conf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL);
TServerTransport serverTransport;
if (forceBindIP != null) {
InetSocketAddress address = new InetSocketAddress(forceBindIP, port);
serverTransport =
tcpKeepAlive ? new TServerSocketKeepAlive(address) : new TServerSocket(address);
} else {
serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port);
}
TProcessor processor;
TTransportFactory transFactory;
IHMSHandler handler = (IHMSHandler) HiveMetaStore
.newRetryingHMSHandler("new db based metaserver",
conf, true);
if (conf.getBoolVar(HiveConf.ConfVars.METASTORE_EXECUTE_SET_UGI)) {
transFactory =
useFramedTransport ? new ChainedTTransportFactory(new TFramedTransport.Factory(),
new TUGIContainingTransport.Factory()) : new TUGIContainingTransport.Factory();
processor = new TUGIBasedProcessor<IHMSHandler>(handler);
LOG.info("Starting DB backed MetaStore Server with SetUGI enabled");
} else {
transFactory =
useFramedTransport ? new TFramedTransport.Factory() : new TTransportFactory();
processor = new TSetIpAddressProcessor<IHMSHandler>(handler);
LOG.info("Starting DB backed MetaStore Server");
}
TThreadPoolServer.Args args = new TThreadPoolServer.Args(serverTransport).processor(processor)
.transportFactory(transFactory)
.protocolFactory(new TBinaryProtocol.Factory())
.minWorkerThreads(minWorkerThreads)
.maxWorkerThreads(maxWorkerThreads);
final TServer tServer = new TThreadPoolServer(args);
executorService.submit(new Runnable() {
@Override
public void run() {
tServer.serve();
}
});
return tServer;
} catch (Throwable x) {
throw new IOException(x);
}
}
}

View File

@@ -0,0 +1,23 @@
###
# 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.
###
log4j.rootLogger=WARN, A1
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
log4j.appender.A1.layout.ConversionPattern=[%-5p] %d %c %x - %m%n

View File

@@ -0,0 +1,26 @@
###
# 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.
###
log4j.rootLogger=WARN, A1
log4j.category.com.uber=INFO
log4j.category.org.apache.parquet.hadoop=WARN
log4j.category.parquet.hadoop=WARN
# A1 is set to be a ConsoleAppender.
log4j.appender.A1=org.apache.log4j.ConsoleAppender
# A1 uses PatternLayout.
log4j.appender.A1.layout=org.apache.log4j.PatternLayout
log4j.appender.A1.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n