CodeStyle formatting to conform to basic Checkstyle rules.
The code-style rules follow google style with some changes: 1. Increase line length from 100 to 120 2. Disable JavaDoc related checkstyles as this needs more manual work. Both source and test code are checked for code-style
This commit is contained in:
committed by
vinoth chandar
parent
987f5d6b96
commit
788e4f2d2e
@@ -48,15 +48,23 @@ public class HiveSyncConfig implements Serializable {
|
||||
"--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 = true)
|
||||
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by",
|
||||
required = true)
|
||||
public List<String> partitionFields = new ArrayList<>();
|
||||
|
||||
@Parameter(names = "-partition-value-extractor", description = "Class which implements PartitionValueExtractor to extract the partition values from HDFS path")
|
||||
@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")
|
||||
"--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 = {"--help", "-h"}, help = true)
|
||||
|
||||
@@ -45,16 +45,16 @@ 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 Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(HiveSyncTool.class);
|
||||
private final HoodieHiveClient hoodieHiveClient;
|
||||
public final static String SUFFIX_REALTIME_TABLE = "_rt";
|
||||
public static final String SUFFIX_REALTIME_TABLE = "_rt";
|
||||
private final HiveSyncConfig cfg;
|
||||
|
||||
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||
@@ -85,9 +85,8 @@ public class HiveSyncTool {
|
||||
}
|
||||
|
||||
private void syncHoodieTable(boolean isRealTime) {
|
||||
LOG.info("Trying to sync hoodie table " + cfg.tableName + " with base path " + hoodieHiveClient
|
||||
.getBasePath() + " of type " + hoodieHiveClient
|
||||
.getTableType());
|
||||
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();
|
||||
@@ -118,27 +117,29 @@ public class HiveSyncTool {
|
||||
* If not, evolves the table schema.
|
||||
*
|
||||
* @param tableExists - does table exist
|
||||
* @param schema - extracted schema
|
||||
* @param schema - extracted schema
|
||||
*/
|
||||
private void syncSchema(boolean tableExists, boolean isRealTime, MessageType schema) {
|
||||
// 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)
|
||||
// TODO - RO Table for MOR only after major compaction (UnboundedCompaction is default
|
||||
// for now)
|
||||
hoodieHiveClient.createTable(schema, HoodieInputFormat.class.getName(),
|
||||
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
|
||||
} 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
|
||||
// https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive
|
||||
// /ql/exec/DDLTask.java#L3488
|
||||
hoodieHiveClient.createTable(schema, HoodieRealtimeInputFormat.class.getName(),
|
||||
MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName());
|
||||
}
|
||||
} else {
|
||||
// Check if the dataset schema has evolved
|
||||
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema();
|
||||
SchemaDifference schemaDiff = SchemaUtil
|
||||
.getSchemaDifference(schema, tableSchema, cfg.partitionFields);
|
||||
SchemaDifference schemaDiff = SchemaUtil.getSchemaDifference(schema, tableSchema,
|
||||
cfg.partitionFields);
|
||||
if (!schemaDiff.isEmpty()) {
|
||||
LOG.info("Schema difference found for " + cfg.tableName);
|
||||
hoodieHiveClient.updateTableDefinition(schema);
|
||||
@@ -156,8 +157,8 @@ public class HiveSyncTool {
|
||||
private void syncPartitions(List<String> writtenPartitionsSince) {
|
||||
try {
|
||||
List<Partition> hivePartitions = hoodieHiveClient.scanTablePartitions();
|
||||
List<PartitionEvent> partitionEvents = hoodieHiveClient
|
||||
.getPartitionEvents(hivePartitions, writtenPartitionsSince);
|
||||
List<PartitionEvent> partitionEvents = hoodieHiveClient.getPartitionEvents(hivePartitions,
|
||||
writtenPartitionsSince);
|
||||
List<String> newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD);
|
||||
LOG.info("New Partitions " + newPartitions);
|
||||
hoodieHiveClient.addPartitionsToTable(newPartitions);
|
||||
@@ -165,15 +166,13 @@ public class HiveSyncTool {
|
||||
LOG.info("Changed Partitions " + updatePartitions);
|
||||
hoodieHiveClient.updatePartitionsToTable(updatePartitions);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieHiveSyncException("Failed to sync partitions for table " + cfg.tableName,
|
||||
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());
|
||||
return events.stream().filter(s -> s.eventType == eventType).map(s -> s.storagePartition)
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws Exception {
|
||||
|
||||
@@ -15,6 +15,7 @@
|
||||
*
|
||||
*
|
||||
*/
|
||||
|
||||
package com.uber.hoodie.hive;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
@@ -25,15 +26,22 @@ import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
import com.uber.hoodie.common.table.HoodieTimeline;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import com.uber.hoodie.exception.InvalidDatasetException;
|
||||
import com.uber.hoodie.hive.util.SchemaUtil;
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.dbcp.BasicDataSource;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -51,18 +59,6 @@ import parquet.hadoop.ParquetFileReader;
|
||||
import parquet.hadoop.metadata.ParquetMetadata;
|
||||
import parquet.schema.MessageType;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.sql.Connection;
|
||||
import java.sql.DatabaseMetaData;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@SuppressWarnings("ConstantConditions")
|
||||
public class HoodieHiveClient {
|
||||
|
||||
@@ -103,16 +99,15 @@ public class HoodieHiveClient {
|
||||
}
|
||||
|
||||
try {
|
||||
this.partitionValueExtractor = (PartitionValueExtractor) Class
|
||||
.forName(cfg.partitionValueExtractorClass).newInstance();
|
||||
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();
|
||||
activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
}
|
||||
|
||||
public HoodieTimeline getActiveTimeline() {
|
||||
@@ -207,9 +202,8 @@ public class HoodieHiveClient {
|
||||
for (Partition tablePartition : tablePartitions) {
|
||||
List<String> hivePartitionValues = tablePartition.getValues();
|
||||
Collections.sort(hivePartitionValues);
|
||||
String fullTablePartitionPath = Path
|
||||
.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getSd().getLocation())).toUri()
|
||||
.getPath();
|
||||
String fullTablePartitionPath = Path.getPathWithoutSchemeAndAuthority(
|
||||
new Path(tablePartition.getSd().getLocation())).toUri().getPath();
|
||||
paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath);
|
||||
}
|
||||
|
||||
@@ -235,8 +229,7 @@ public class HoodieHiveClient {
|
||||
* Scan table partitions
|
||||
*/
|
||||
List<Partition> scanTablePartitions() throws TException {
|
||||
return client
|
||||
.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1);
|
||||
return client.listPartitions(syncConfig.databaseName, syncConfig.tableName, (short) -1);
|
||||
}
|
||||
|
||||
void updateTableDefinition(MessageType newSchema) {
|
||||
@@ -245,9 +238,10 @@ public class HoodieHiveClient {
|
||||
// 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);
|
||||
.append(syncConfig.databaseName).append(".")
|
||||
.append(syncConfig.tableName).append("`")
|
||||
.append(" REPLACE COLUMNS(").append(newSchemaStr).append(" )")
|
||||
.append(cascadeClause);
|
||||
LOG.info("Creating table with " + sqlBuilder);
|
||||
updateHiveSQL(sqlBuilder.toString());
|
||||
} catch (IOException e) {
|
||||
@@ -255,8 +249,8 @@ public class HoodieHiveClient {
|
||||
}
|
||||
}
|
||||
|
||||
void createTable(MessageType storageSchema,
|
||||
String inputFormatClass, String outputFormatClass, String serdeClass) {
|
||||
void createTable(MessageType storageSchema, String inputFormatClass, String outputFormatClass,
|
||||
String serdeClass) {
|
||||
try {
|
||||
String createSQLQuery = SchemaUtil
|
||||
.generateCreateDDL(storageSchema, syncConfig, inputFormatClass,
|
||||
@@ -289,8 +283,8 @@ public class HoodieHiveClient {
|
||||
}
|
||||
return schema;
|
||||
} catch (SQLException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get table schema for " + syncConfig.tableName, e);
|
||||
throw new HoodieHiveSyncException("Failed to get table schema for " + syncConfig.tableName,
|
||||
e);
|
||||
} finally {
|
||||
closeQuietly(result, null);
|
||||
}
|
||||
@@ -308,49 +302,47 @@ public class HoodieHiveClient {
|
||||
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());
|
||||
// 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());
|
||||
String filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
|
||||
.stream().findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
.stream().findAny().orElseThrow(() -> new IllegalArgumentException(
|
||||
"Could not find any data file written for commit " + lastCommit
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
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
|
||||
// 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
|
||||
Optional<HoodieInstant> lastCompactionCommit = metaClient.getActiveTimeline()
|
||||
.getCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
.getCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
.lastInstant();
|
||||
LOG.info("Found the last compaction commit as " + lastCompactionCommit);
|
||||
|
||||
Optional<HoodieInstant> lastDeltaCommit;
|
||||
if (lastCompactionCommit.isPresent()) {
|
||||
lastDeltaCommit = metaClient.getActiveTimeline()
|
||||
.getDeltaCommitTimeline()
|
||||
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(), Integer.MAX_VALUE)
|
||||
.lastInstant();
|
||||
.findInstantsAfter(lastCompactionCommit.get().getTimestamp(),
|
||||
Integer.MAX_VALUE).lastInstant();
|
||||
} else {
|
||||
lastDeltaCommit = metaClient.getActiveTimeline()
|
||||
.getDeltaCommitTimeline()
|
||||
.filterCompletedInstants()
|
||||
.lastInstant();
|
||||
lastDeltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline()
|
||||
.filterCompletedInstants().lastInstant();
|
||||
}
|
||||
LOG.info("Found the last delta commit "
|
||||
+ lastDeltaCommit);
|
||||
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());
|
||||
commitMetadata = HoodieCommitMetadata.fromBytes(
|
||||
activeTimeline.getInstantDetails(lastDeltaInstant).get());
|
||||
filePath = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
|
||||
.stream().filter(s -> s.contains(
|
||||
HoodieLogFile.DELTA_EXTENSION)).findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
.stream().filter(s -> s.contains(HoodieLogFile.DELTA_EXTENSION))
|
||||
.findAny().orElseThrow(() -> new IllegalArgumentException(
|
||||
"Could not find any data file written for commit " + lastDeltaInstant
|
||||
+ ", could not get schema for dataset " + metaClient.getBasePath()));
|
||||
return readSchemaFromLogFile(lastCompactionCommit, new Path(filePath));
|
||||
@@ -362,8 +354,8 @@ public class HoodieHiveClient {
|
||||
throw new InvalidDatasetException(syncConfig.basePath);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to get dataset schema for " + syncConfig.tableName, e);
|
||||
throw new HoodieHiveSyncException("Failed to get dataset schema for " + syncConfig.tableName,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -379,11 +371,10 @@ public class HoodieHiveClient {
|
||||
+ syncConfig.basePath));
|
||||
|
||||
// Read from the compacted file wrote
|
||||
HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata
|
||||
.fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get());
|
||||
HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata.fromBytes(
|
||||
activeTimeline.getInstantDetails(lastCompactionCommit).get());
|
||||
String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values()
|
||||
.stream().findAny()
|
||||
.orElseThrow(() -> new IllegalArgumentException(
|
||||
.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));
|
||||
@@ -411,12 +402,10 @@ public class HoodieHiveClient {
|
||||
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.");
|
||||
"Failed to read schema from data file " + parquetFilePath + ". File does not exist.");
|
||||
}
|
||||
ParquetMetadata fileFooter =
|
||||
ParquetFileReader
|
||||
.readFooter(fs.getConf(), parquetFilePath, ParquetMetadataConverter.NO_FILTER);
|
||||
ParquetMetadata fileFooter = ParquetFileReader.readFooter(fs.getConf(), parquetFilePath,
|
||||
ParquetMetadataConverter.NO_FILTER);
|
||||
return fileFooter.getFileMetaData().getSchema();
|
||||
}
|
||||
|
||||
@@ -427,8 +416,8 @@ public class HoodieHiveClient {
|
||||
try {
|
||||
return client.tableExists(syncConfig.databaseName, syncConfig.tableName);
|
||||
} catch (TException e) {
|
||||
throw new HoodieHiveSyncException(
|
||||
"Failed to check if table exists " + syncConfig.tableName, e);
|
||||
throw new HoodieHiveSyncException("Failed to check if table exists " + syncConfig.tableName,
|
||||
e);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -511,8 +500,8 @@ public class HoodieHiveClient {
|
||||
// Get the last commit time from the TBLproperties
|
||||
try {
|
||||
Table database = client.getTable(syncConfig.databaseName, syncConfig.tableName);
|
||||
return Optional
|
||||
.ofNullable(database.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null));
|
||||
return Optional.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);
|
||||
@@ -537,8 +526,8 @@ public class HoodieHiveClient {
|
||||
if (!lastCommitTimeSynced.isPresent()) {
|
||||
LOG.info("Last commit time synced is not known, listing all partitions");
|
||||
try {
|
||||
return FSUtils
|
||||
.getAllPartitionPaths(fs, syncConfig.basePath, syncConfig.assumeDatePartitioning);
|
||||
return FSUtils.getAllPartitionPaths(fs, syncConfig.basePath,
|
||||
syncConfig.assumeDatePartitioning);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Failed to list all partitions in " + syncConfig.basePath, e);
|
||||
}
|
||||
@@ -546,8 +535,8 @@ public class HoodieHiveClient {
|
||||
LOG.info("Last commit time synced is " + lastCommitTimeSynced.get()
|
||||
+ ", Getting commits since then");
|
||||
|
||||
HoodieTimeline timelineToSync = activeTimeline
|
||||
.findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE);
|
||||
HoodieTimeline timelineToSync = activeTimeline.findInstantsAfter(lastCommitTimeSynced.get(),
|
||||
Integer.MAX_VALUE);
|
||||
return timelineToSync.getInstants().map(s -> {
|
||||
try {
|
||||
return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(s).get());
|
||||
@@ -579,13 +568,15 @@ public class HoodieHiveClient {
|
||||
*/
|
||||
static class PartitionEvent {
|
||||
|
||||
public enum PartitionEventType {ADD, UPDATE}
|
||||
public enum PartitionEventType {
|
||||
ADD,
|
||||
UPDATE
|
||||
}
|
||||
|
||||
PartitionEventType eventType;
|
||||
String storagePartition;
|
||||
|
||||
PartitionEvent(
|
||||
PartitionEventType eventType, String storagePartition) {
|
||||
PartitionEvent(PartitionEventType eventType, String storagePartition) {
|
||||
this.eventType = eventType;
|
||||
this.storagePartition = storagePartition;
|
||||
}
|
||||
|
||||
@@ -24,8 +24,9 @@ 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.
|
||||
*
|
||||
* e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd]
|
||||
* <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 {
|
||||
|
||||
|
||||
@@ -28,7 +28,7 @@ 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 {
|
||||
|
||||
@@ -26,9 +26,9 @@ public class ColumnNameXLator {
|
||||
|
||||
public static String translateNestedColumn(String colName) {
|
||||
Map.Entry entry;
|
||||
for (Iterator i$ = xformMap.entrySet().iterator(); i$.hasNext();
|
||||
for (Iterator ic = xformMap.entrySet().iterator(); ic.hasNext();
|
||||
colName = colName.replaceAll((String) entry.getKey(), (String) entry.getValue())) {
|
||||
entry = (Map.Entry) i$.next();
|
||||
entry = (Map.Entry) ic.next();
|
||||
}
|
||||
|
||||
return colName;
|
||||
|
||||
@@ -23,14 +23,12 @@ import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.Reader;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.hive.HiveSyncConfig;
|
||||
import com.uber.hoodie.hive.HoodieHiveSyncException;
|
||||
import com.uber.hoodie.hive.SchemaDifference;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -48,7 +46,7 @@ import parquet.schema.Type;
|
||||
*/
|
||||
public class SchemaUtil {
|
||||
|
||||
private static Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
|
||||
private static final Logger LOG = LoggerFactory.getLogger(SchemaUtil.class);
|
||||
|
||||
/**
|
||||
* Get the schema difference between the storage schema and hive table schema
|
||||
@@ -59,19 +57,19 @@ public class SchemaUtil {
|
||||
try {
|
||||
newTableSchema = convertParquetSchemaToHiveSchema(storageSchema);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieHiveSyncException("Failed to convert parquet schema to hive schema",
|
||||
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);
|
||||
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)) {
|
||||
.contains(
|
||||
fieldName)) {
|
||||
schemaDiffBuilder.deleteTableColumn(fieldName);
|
||||
} else {
|
||||
// check type
|
||||
@@ -82,8 +80,8 @@ public class SchemaUtil {
|
||||
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");
|
||||
LOG.warn(
|
||||
"Ignoring table column " + fieldName + " as its not present in the parquet schema");
|
||||
continue;
|
||||
}
|
||||
tableColumnType = tableColumnType.replaceAll("\\s+", "");
|
||||
@@ -93,11 +91,12 @@ public class SchemaUtil {
|
||||
expectedType = expectedType.replaceAll("`", "");
|
||||
|
||||
if (!tableColumnType.equalsIgnoreCase(expectedType)) {
|
||||
// check for incremental datasets, the schema type change is allowed as per evolution rules
|
||||
// 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);
|
||||
"Could not convert field Type from " + tableColumnType + " to " + expectedType
|
||||
+ " for field " + fieldName);
|
||||
}
|
||||
schemaDiffBuilder.updateTableColumn(fieldName,
|
||||
getExpectedType(newTableSchema, tickSurroundedFieldName));
|
||||
@@ -169,21 +168,19 @@ public class SchemaUtil {
|
||||
private static String convertField(final Type parquetType) {
|
||||
StringBuilder field = new StringBuilder();
|
||||
if (parquetType.isPrimitive()) {
|
||||
final PrimitiveType.PrimitiveTypeName parquetPrimitiveTypeName =
|
||||
parquetType.asPrimitiveType().getPrimitiveTypeName();
|
||||
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();
|
||||
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) {
|
||||
public String convertBOOLEAN(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
return "boolean";
|
||||
}
|
||||
|
||||
@@ -220,8 +217,7 @@ public class SchemaUtil {
|
||||
|
||||
@Override
|
||||
public String convertBINARY(PrimitiveType.PrimitiveTypeName primitiveTypeName) {
|
||||
if (originalType == OriginalType.UTF8
|
||||
|| originalType == OriginalType.ENUM) {
|
||||
if (originalType == OriginalType.UTF8 || originalType == OriginalType.ENUM) {
|
||||
return "string";
|
||||
} else {
|
||||
return "binary";
|
||||
@@ -235,33 +231,28 @@ public class SchemaUtil {
|
||||
switch (originalType) {
|
||||
case LIST:
|
||||
if (parquetGroupType.getFieldCount() != 1) {
|
||||
throw new UnsupportedOperationException(
|
||||
"Invalid list type " + parquetGroupType);
|
||||
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);
|
||||
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);
|
||||
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);
|
||||
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)) {
|
||||
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);
|
||||
}
|
||||
@@ -277,8 +268,7 @@ public class SchemaUtil {
|
||||
// MAP. Hence, PARQUET-113
|
||||
// dropped the requirement for having MAP_KEY_VALUE.
|
||||
default:
|
||||
throw new UnsupportedOperationException(
|
||||
"Cannot convert Parquet type " + parquetType);
|
||||
throw new UnsupportedOperationException("Cannot convert Parquet type " + parquetType);
|
||||
}
|
||||
} else {
|
||||
// if no original type then it's a record
|
||||
@@ -306,7 +296,8 @@ public class SchemaUtil {
|
||||
// ", "
|
||||
struct.append(">");
|
||||
String finalStr = struct.toString();
|
||||
// Struct cannot have - in them. userstore_udr_entities has uuid in struct. This breaks the schema.
|
||||
// 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;
|
||||
@@ -362,8 +353,8 @@ public class SchemaUtil {
|
||||
}
|
||||
|
||||
public static boolean isSchemaTypeUpdateAllowed(String prevType, String newType) {
|
||||
if (prevType == null || prevType.trim().isEmpty() ||
|
||||
newType == null || newType.trim().isEmpty()) {
|
||||
if (prevType == null || prevType.trim().isEmpty() || newType == null || newType.trim()
|
||||
.isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
prevType = prevType.toLowerCase();
|
||||
@@ -392,9 +383,8 @@ public class SchemaUtil {
|
||||
return columns.toString();
|
||||
}
|
||||
|
||||
public static String generateCreateDDL(MessageType storageSchema,
|
||||
HiveSyncConfig config, String inputFormatClass,
|
||||
String outputFormatClass, String serdeClass) throws IOException {
|
||||
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);
|
||||
|
||||
@@ -422,7 +412,8 @@ public class SchemaUtil {
|
||||
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
|
||||
// TODO - all partition fields should be part of the schema. datestr is treated as special.
|
||||
// Dont do that
|
||||
return "String";
|
||||
}
|
||||
|
||||
@@ -430,8 +421,7 @@ public class SchemaUtil {
|
||||
* Read the schema from the log file on path
|
||||
*/
|
||||
@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
|
||||
public static MessageType readSchemaFromLogFile(FileSystem fs,
|
||||
Path path) throws IOException {
|
||||
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()) {
|
||||
|
||||
@@ -60,39 +60,39 @@ public class HiveSyncToolTest {
|
||||
@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");
|
||||
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");
|
||||
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");
|
||||
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");
|
||||
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);
|
||||
@@ -100,10 +100,10 @@ public class HiveSyncToolTest {
|
||||
// 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");
|
||||
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);
|
||||
@@ -111,10 +111,10 @@ public class HiveSyncToolTest {
|
||||
// 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");
|
||||
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);
|
||||
@@ -122,22 +122,22 @@ public class HiveSyncToolTest {
|
||||
// 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");
|
||||
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");
|
||||
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);
|
||||
@@ -146,7 +146,8 @@ public class HiveSyncToolTest {
|
||||
|
||||
@Test
|
||||
public void testBasicSync()
|
||||
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
|
||||
throws IOException, InitializationError, URISyntaxException, TException,
|
||||
InterruptedException {
|
||||
String commitTime = "100";
|
||||
TestUtil.createCOWDataset(commitTime, 5);
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
@@ -160,18 +161,17 @@ public class HiveSyncToolTest {
|
||||
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);
|
||||
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());
|
||||
commitTime, hiveClient.getLastCommitTimeSynced().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncIncremental()
|
||||
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
|
||||
throws IOException, InitializationError, URISyntaxException, TException,
|
||||
InterruptedException {
|
||||
String commitTime1 = "100";
|
||||
TestUtil.createCOWDataset(commitTime1, 5);
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
@@ -183,8 +183,7 @@ public class HiveSyncToolTest {
|
||||
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());
|
||||
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);
|
||||
@@ -192,33 +191,32 @@ public class HiveSyncToolTest {
|
||||
TestUtil.addCOWPartitions(1, true, dateTime, commitTime2);
|
||||
|
||||
// Lets do the sync
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
List<String> writtenPartitionsSince = hiveClient
|
||||
.getPartitionsWrittenToSince(Optional.of(commitTime1));
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
TestUtil.fileSystem);
|
||||
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(
|
||||
Optional.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);
|
||||
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 = 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,
|
||||
assertEquals("The last commit that was sycned should be 101", commitTime2,
|
||||
hiveClient.getLastCommitTimeSynced().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncIncrementalWithSchemaEvolution()
|
||||
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
|
||||
throws IOException, InitializationError, URISyntaxException, TException,
|
||||
InterruptedException {
|
||||
String commitTime1 = "100";
|
||||
TestUtil.createCOWDataset(commitTime1, 5);
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
@@ -236,30 +234,27 @@ public class HiveSyncToolTest {
|
||||
TestUtil.addCOWPartitions(1, false, dateTime, commitTime2);
|
||||
|
||||
// Lets do the sync
|
||||
tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
TestUtil.fileSystem);
|
||||
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,
|
||||
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"));
|
||||
"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,
|
||||
assertEquals("The last commit that was sycned should be 101", commitTime2,
|
||||
hiveClient.getLastCommitTimeSynced().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncMergeOnRead()
|
||||
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
|
||||
throws IOException, InitializationError, URISyntaxException, TException,
|
||||
InterruptedException {
|
||||
String commitTime = "100";
|
||||
String deltaCommitTime = "101";
|
||||
TestUtil.createMORDataset(commitTime, deltaCommitTime, 5);
|
||||
@@ -280,8 +275,7 @@ public class HiveSyncToolTest {
|
||||
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());
|
||||
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);
|
||||
@@ -291,11 +285,10 @@ public class HiveSyncToolTest {
|
||||
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 = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
hiveClient = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
TestUtil.fileSystem);
|
||||
|
||||
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
|
||||
hiveClient.getTableSchema().size(),
|
||||
@@ -303,14 +296,14 @@ public class HiveSyncToolTest {
|
||||
// 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,
|
||||
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
|
||||
hiveClient.getLastCommitTimeSynced().get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSyncMergeOnReadRT()
|
||||
throws IOException, InitializationError, URISyntaxException, TException, InterruptedException {
|
||||
throws IOException, InitializationError, URISyntaxException, TException,
|
||||
InterruptedException {
|
||||
String commitTime = "100";
|
||||
String deltaCommitTime = "101";
|
||||
String roTablename = TestUtil.hiveSyncConfig.tableName;
|
||||
@@ -321,8 +314,7 @@ public class HiveSyncToolTest {
|
||||
TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
|
||||
assertFalse("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE
|
||||
+ " should not exist initially",
|
||||
hiveClientRT.doesTableExist());
|
||||
+ " should not exist initially", hiveClientRT.doesTableExist());
|
||||
|
||||
// Lets do the sync
|
||||
HiveSyncTool tool = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
@@ -330,8 +322,7 @@ public class HiveSyncToolTest {
|
||||
tool.syncHoodieTable();
|
||||
|
||||
assertTrue("Table " + TestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_REALTIME_TABLE
|
||||
+ " should exist after sync completes",
|
||||
hiveClientRT.doesTableExist());
|
||||
+ " should exist after sync completes", hiveClientRT.doesTableExist());
|
||||
|
||||
assertEquals("Hive Schema should match the dataset schema + partition field",
|
||||
hiveClientRT.getTableSchema().size(),
|
||||
@@ -339,8 +330,7 @@ public class HiveSyncToolTest {
|
||||
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());
|
||||
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);
|
||||
@@ -350,11 +340,10 @@ public class HiveSyncToolTest {
|
||||
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 = new HiveSyncTool(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig,
|
||||
TestUtil.getHiveConf(), TestUtil.fileSystem);
|
||||
hiveClientRT = new HoodieHiveClient(TestUtil.hiveSyncConfig, TestUtil.getHiveConf(),
|
||||
TestUtil.fileSystem);
|
||||
|
||||
assertEquals("Hive Schema should match the evolved dataset schema + partition field",
|
||||
hiveClientRT.getTableSchema().size(),
|
||||
@@ -362,8 +351,7 @@ public class HiveSyncToolTest {
|
||||
// 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,
|
||||
assertEquals("The last commit that was sycned should be 103", deltaCommitTime2,
|
||||
hiveClientRT.getLastCommitTimeSynced().get());
|
||||
TestUtil.hiveSyncConfig.tableName = roTablename;
|
||||
}
|
||||
|
||||
@@ -16,6 +16,9 @@
|
||||
|
||||
package com.uber.hoodie.hive;
|
||||
|
||||
import static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
@@ -39,6 +42,15 @@ import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.SchemaTestUtil;
|
||||
import com.uber.hoodie.hive.util.HiveTestService;
|
||||
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;
|
||||
@@ -58,19 +70,6 @@ import org.joda.time.format.DateTimeFormat;
|
||||
import org.joda.time.format.DateTimeFormatter;
|
||||
import org.junit.runners.model.InitializationError;
|
||||
|
||||
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 static com.uber.hoodie.common.model.HoodieTestUtils.DEFAULT_TASK_PARTITIONID;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
@SuppressWarnings("SameParameterValue")
|
||||
public class TestUtil {
|
||||
|
||||
@@ -127,8 +126,7 @@ public class TestUtil {
|
||||
client.updateHiveSQL("drop table if exists " + tableName);
|
||||
}
|
||||
createdTablesSet.clear();
|
||||
client.updateHiveSQL(
|
||||
"drop database if exists " + hiveSyncConfig.databaseName);
|
||||
client.updateHiveSQL("drop database if exists " + hiveSyncConfig.databaseName);
|
||||
client.updateHiveSQL("create database " + hiveSyncConfig.databaseName);
|
||||
}
|
||||
|
||||
@@ -182,9 +180,8 @@ public class TestUtil {
|
||||
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)));
|
||||
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(),
|
||||
@@ -202,8 +199,7 @@ public class TestUtil {
|
||||
}
|
||||
|
||||
static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple,
|
||||
boolean isLogSchemaSimple, DateTime startFrom,
|
||||
String commitTime, String deltaCommitTime)
|
||||
boolean isLogSchemaSimple, DateTime startFrom, String commitTime, String deltaCommitTime)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions,
|
||||
isParquetSchemaSimple, startFrom, commitTime);
|
||||
@@ -211,9 +207,8 @@ public class TestUtil {
|
||||
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)));
|
||||
commitMetadata.getPartitionToWriteStats().forEach(
|
||||
(key, value) -> value.stream().forEach(l -> compactionMetadata.addWriteStat(key, l)));
|
||||
createCompactionCommitFile(compactionMetadata, commitTime);
|
||||
HoodieCommitMetadata deltaMetadata = createLogFiles(commitMetadata.getPartitionToWriteStats(),
|
||||
isLogSchemaSimple);
|
||||
@@ -264,8 +259,8 @@ public class TestUtil {
|
||||
for (int i = 0; i < 5; i++) {
|
||||
// Create 5 files
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
Path filePath = new Path(partPath.toString() + "/" + FSUtils
|
||||
.makeDataFileName(commitTime, DEFAULT_TASK_PARTITIONID, fileId));
|
||||
Path filePath = new Path(partPath.toString() + "/" + FSUtils.makeDataFileName(commitTime,
|
||||
DEFAULT_TASK_PARTITIONID, fileId));
|
||||
generateParquetData(filePath, isParquetSchemaSimple);
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId(fileId);
|
||||
@@ -283,11 +278,10 @@ public class TestUtil {
|
||||
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());
|
||||
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)
|
||||
@@ -309,10 +303,10 @@ public class TestUtil {
|
||||
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())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION)
|
||||
.withFileId(dataFile.getFileId())
|
||||
.overBaseCommit(dataFile.getCommitTime()).withFs(fileSystem).build();
|
||||
List<IndexedRecord> records = (isLogSchemaSimple ? SchemaTestUtil
|
||||
.generateTestRecords(0, 100)
|
||||
List<IndexedRecord> records = (isLogSchemaSimple ? SchemaTestUtil.generateTestRecords(0, 100)
|
||||
: SchemaTestUtil.generateEvolvedTestRecords(100, 100));
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
|
||||
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, dataFile.getCommitTime());
|
||||
@@ -329,37 +323,37 @@ public class TestUtil {
|
||||
}
|
||||
}
|
||||
|
||||
private static void createCommitFile(
|
||||
HoodieCommitMetadata commitMetadata, String commitTime)
|
||||
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));
|
||||
.makeCommitFileName(
|
||||
commitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
}
|
||||
|
||||
private static void createCompactionCommitFile(
|
||||
HoodieCommitMetadata commitMetadata, String commitTime)
|
||||
throws IOException {
|
||||
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));
|
||||
.makeCommitFileName(
|
||||
commitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
}
|
||||
|
||||
private static void createDeltaCommitFile(
|
||||
HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime)
|
||||
throws IOException {
|
||||
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));
|
||||
.makeDeltaFileName(
|
||||
deltaCommitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
fsout.close();
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.hive.util;
|
||||
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.io.Files;
|
||||
@@ -84,8 +83,7 @@ public class HiveTestService {
|
||||
}
|
||||
|
||||
public HiveServer2 start() throws IOException {
|
||||
Preconditions
|
||||
.checkState(workDir != null, "The work dir must be set before starting cluster.");
|
||||
Preconditions.checkState(workDir != null, "The work dir must be set before starting cluster.");
|
||||
|
||||
if (hadoopConf == null) {
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
@@ -93,8 +91,7 @@ public class HiveTestService {
|
||||
|
||||
String localHiveLocation = getHiveLocation(workDir);
|
||||
if (clean) {
|
||||
LOG.info(
|
||||
"Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh.");
|
||||
LOG.info("Cleaning Hive cluster data at: " + localHiveLocation + " and starting fresh.");
|
||||
File file = new File(localHiveLocation);
|
||||
FileUtils.deleteDirectory(file);
|
||||
}
|
||||
@@ -134,11 +131,9 @@ public class HiveTestService {
|
||||
hadoopConf = null;
|
||||
}
|
||||
|
||||
private HiveConf configureHive(Configuration conf, String localHiveLocation)
|
||||
throws IOException {
|
||||
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.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
|
||||
@@ -154,8 +149,7 @@ public class HiveTestService {
|
||||
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(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, Files.createTempDir().getAbsolutePath());
|
||||
|
||||
return new HiveConf(conf, this.getClass());
|
||||
}
|
||||
@@ -269,8 +263,8 @@ public class HiveTestService {
|
||||
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);
|
||||
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);
|
||||
@@ -282,21 +276,20 @@ public class HiveTestService {
|
||||
tcpKeepAlive ? new TServerSocketKeepAlive(address) : new TServerSocket(address);
|
||||
|
||||
} else {
|
||||
serverTransport =
|
||||
tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port);
|
||||
serverTransport = tcpKeepAlive ? new TServerSocketKeepAlive(port) : new TServerSocket(port);
|
||||
}
|
||||
|
||||
TProcessor processor;
|
||||
TTransportFactory transFactory;
|
||||
|
||||
IHMSHandler handler = (IHMSHandler) HiveMetaStore
|
||||
.newRetryingHMSHandler("new db based metaserver", conf, true);
|
||||
.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();
|
||||
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");
|
||||
@@ -307,10 +300,11 @@ public class HiveTestService {
|
||||
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);
|
||||
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() {
|
||||
|
||||
Reference in New Issue
Block a user