[HUDI-3507] Support export command based on Call Produce Command (#5901)
This commit is contained in:
@@ -18,6 +18,12 @@
|
||||
|
||||
package org.apache.hudi.cli.commands;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.specific.SpecificData;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.model.HoodieArchivedMetaEntry;
|
||||
import org.apache.hudi.avro.model.HoodieCleanMetadata;
|
||||
@@ -36,14 +42,6 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.avro.specific.SpecificData;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.springframework.shell.core.CommandMarker;
|
||||
import org.springframework.shell.core.annotation.CliCommand;
|
||||
import org.springframework.shell.core.annotation.CliOption;
|
||||
@@ -60,10 +58,10 @@ import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* CLI commands to export various information from a HUDI dataset.
|
||||
*
|
||||
* <p>
|
||||
* "export instants": Export Instants and their metadata from the Timeline to a local
|
||||
* directory specified by the parameter --localFolder
|
||||
* The instants are exported in the json format.
|
||||
* directory specified by the parameter --localFolder
|
||||
* The instants are exported in the json format.
|
||||
*/
|
||||
@Component
|
||||
public class ExportCommand implements CommandMarker {
|
||||
@@ -83,7 +81,7 @@ public class ExportCommand implements CommandMarker {
|
||||
int numExports = limit == -1 ? Integer.MAX_VALUE : limit;
|
||||
int numCopied = 0;
|
||||
|
||||
if (! new File(localFolder).isDirectory()) {
|
||||
if (!new File(localFolder).isDirectory()) {
|
||||
throw new HoodieException(localFolder + " is not a valid local directory");
|
||||
}
|
||||
|
||||
@@ -94,7 +92,7 @@ public class ExportCommand implements CommandMarker {
|
||||
|
||||
// Archived instants are in the commit archive files
|
||||
FileStatus[] statuses = FSUtils.getFs(basePath, HoodieCLI.conf).globStatus(archivePath);
|
||||
List<FileStatus> archivedStatuses = Arrays.stream(statuses).sorted((f1, f2) -> (int)(f1.getModificationTime() - f2.getModificationTime())).collect(Collectors.toList());
|
||||
List<FileStatus> archivedStatuses = Arrays.stream(statuses).sorted((f1, f2) -> (int) (f1.getModificationTime() - f2.getModificationTime())).collect(Collectors.toList());
|
||||
|
||||
if (descending) {
|
||||
Collections.reverse(nonArchivedInstants);
|
||||
@@ -115,11 +113,11 @@ public class ExportCommand implements CommandMarker {
|
||||
|
||||
private int copyArchivedInstants(List<FileStatus> statuses, Set<String> actionSet, int limit, String localFolder) throws Exception {
|
||||
int copyCount = 0;
|
||||
FileSystem fileSystem = FSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf);
|
||||
|
||||
for (FileStatus fs : statuses) {
|
||||
// read the archived file
|
||||
Reader reader = HoodieLogFormat.newReader(FSUtils.getFs(HoodieCLI.getTableMetaClient().getBasePath(), HoodieCLI.conf),
|
||||
new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
Reader reader = HoodieLogFormat.newReader(fileSystem, new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema());
|
||||
|
||||
// read the avro blocks
|
||||
while (reader.hasNext() && copyCount < limit) {
|
||||
@@ -130,7 +128,7 @@ public class ExportCommand implements CommandMarker {
|
||||
// Archived instants are saved as arvo encoded HoodieArchivedMetaEntry records. We need to get the
|
||||
// metadata record from the entry and convert it to json.
|
||||
HoodieArchivedMetaEntry archiveEntryRecord = (HoodieArchivedMetaEntry) SpecificData.get()
|
||||
.deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
|
||||
.deepCopy(HoodieArchivedMetaEntry.SCHEMA$, ir);
|
||||
final String action = archiveEntryRecord.get("actionType").toString();
|
||||
if (!actionSet.contains(action)) {
|
||||
continue;
|
||||
@@ -157,7 +155,7 @@ public class ExportCommand implements CommandMarker {
|
||||
default:
|
||||
throw new HoodieException("Unknown type of action " + action);
|
||||
}
|
||||
|
||||
|
||||
final String instantTime = archiveEntryRecord.get("commitTime").toString();
|
||||
final String outPath = localFolder + Path.SEPARATOR + instantTime + "." + action;
|
||||
writeToFile(outPath, HoodieAvroUtils.avroToJson(metadata, true));
|
||||
@@ -178,9 +176,8 @@ public class ExportCommand implements CommandMarker {
|
||||
int copyCount = 0;
|
||||
|
||||
if (instants.isEmpty()) {
|
||||
return limit;
|
||||
return copyCount;
|
||||
}
|
||||
final Logger LOG = LogManager.getLogger(ExportCommand.class);
|
||||
|
||||
final HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient();
|
||||
final HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
|
||||
@@ -221,6 +218,7 @@ public class ExportCommand implements CommandMarker {
|
||||
|
||||
if (data != null) {
|
||||
writeToFile(localPath, data);
|
||||
copyCount = copyCount + 1;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user