1
0

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:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -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 {