Close Hoodie Clients which are opened to properly shutdown embedded timeline service
This commit is contained in:
committed by
vinoth chandar
parent
065173211e
commit
51d122b5c3
@@ -183,14 +183,19 @@ public class DataSourceUtils {
|
||||
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc,
|
||||
JavaRDD<HoodieRecord> incomingHoodieRecords,
|
||||
HoodieWriteConfig writeConfig) throws Exception {
|
||||
HoodieReadClient client = null;
|
||||
try {
|
||||
HoodieReadClient client = new HoodieReadClient<>(jssc, writeConfig);
|
||||
client = new HoodieReadClient<>(jssc, writeConfig);
|
||||
return client.tagLocation(incomingHoodieRecords)
|
||||
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
|
||||
} catch (DatasetNotFoundException e) {
|
||||
// this will be executed when there is no hoodie dataset yet
|
||||
// so no dups to drop
|
||||
return incomingHoodieRecords;
|
||||
} finally {
|
||||
if (null != client) {
|
||||
client.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -58,47 +58,52 @@ public class HoodieCompactionAdminTool {
|
||||
*/
|
||||
public void run(JavaSparkContext jsc) throws Exception {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(), cfg.basePath);
|
||||
CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath);
|
||||
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
||||
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
|
||||
throw new IllegalStateException("Output File Path already exists");
|
||||
}
|
||||
switch (cfg.operation) {
|
||||
case VALIDATE:
|
||||
List<ValidationOpResult> res =
|
||||
admin.validateCompactionPlan(metaClient, cfg.compactionInstantTime, cfg.parallelism);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Validation Operation :", res);
|
||||
}
|
||||
serializeOperationResult(fs, res);
|
||||
break;
|
||||
case UNSCHEDULE_FILE:
|
||||
List<RenameOpResult> r =
|
||||
admin.unscheduleCompactionFileId(new HoodieFileGroupId(cfg.partitionPath, cfg.fileId),
|
||||
cfg.skipValidation, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
System.out.println(r);
|
||||
}
|
||||
serializeOperationResult(fs, r);
|
||||
break;
|
||||
case UNSCHEDULE_PLAN:
|
||||
List<RenameOpResult> r2 =
|
||||
admin.unscheduleCompactionPlan(cfg.compactionInstantTime, cfg.skipValidation, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Unscheduling Compaction Plan :", r2);
|
||||
}
|
||||
serializeOperationResult(fs, r2);
|
||||
break;
|
||||
case REPAIR:
|
||||
List<RenameOpResult> r3 =
|
||||
admin.repairCompaction(cfg.compactionInstantTime, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Repair Operation :", r3);
|
||||
}
|
||||
serializeOperationResult(fs, r3);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Not yet implemented !!");
|
||||
final CompactionAdminClient admin = new CompactionAdminClient(jsc, cfg.basePath);
|
||||
try {
|
||||
final FileSystem fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration());
|
||||
if (cfg.outputPath != null && fs.exists(new Path(cfg.outputPath))) {
|
||||
throw new IllegalStateException("Output File Path already exists");
|
||||
}
|
||||
switch (cfg.operation) {
|
||||
case VALIDATE:
|
||||
List<ValidationOpResult> res =
|
||||
admin.validateCompactionPlan(metaClient, cfg.compactionInstantTime, cfg.parallelism);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Validation Operation :", res);
|
||||
}
|
||||
serializeOperationResult(fs, res);
|
||||
break;
|
||||
case UNSCHEDULE_FILE:
|
||||
List<RenameOpResult> r =
|
||||
admin.unscheduleCompactionFileId(new HoodieFileGroupId(cfg.partitionPath, cfg.fileId),
|
||||
cfg.skipValidation, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
System.out.println(r);
|
||||
}
|
||||
serializeOperationResult(fs, r);
|
||||
break;
|
||||
case UNSCHEDULE_PLAN:
|
||||
List<RenameOpResult> r2 =
|
||||
admin
|
||||
.unscheduleCompactionPlan(cfg.compactionInstantTime, cfg.skipValidation, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Unscheduling Compaction Plan :", r2);
|
||||
}
|
||||
serializeOperationResult(fs, r2);
|
||||
break;
|
||||
case REPAIR:
|
||||
List<RenameOpResult> r3 =
|
||||
admin.repairCompaction(cfg.compactionInstantTime, cfg.parallelism, cfg.dryRun);
|
||||
if (cfg.printOutput) {
|
||||
printOperationResult("Result of Repair Operation :", r3);
|
||||
}
|
||||
serializeOperationResult(fs, r3);
|
||||
break;
|
||||
default:
|
||||
throw new IllegalStateException("Not yet implemented !!");
|
||||
}
|
||||
} finally {
|
||||
admin.close();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user