in core/src/main/java/org/apache/accumulo/core/file/rfile/PrintInfo.java [164:308]
public void execute(final String[] args) throws Exception {
Opts opts = new Opts();
opts.parseArgs("accumulo rfile-info", args);
if (opts.files.isEmpty()) {
System.err.println("No files were given");
System.exit(1);
}
if ((opts.fullKeys || opts.dump) && opts.formatterClazz != null) {
System.err.println(
"--formatter argument is incompatible with --dump or --fullKeys, specify either, not both.");
System.exit(1);
}
var siteConfig = opts.getSiteConfiguration();
Configuration conf = new Configuration();
for (String confFile : opts.configFiles) {
log.debug("Adding Hadoop configuration file {}", confFile);
conf.addResource(new Path(confFile));
}
LogHistogram kvHistogram = new LogHistogram();
KeyStats dataKeyStats = new KeyStats();
KeyStats indexKeyStats = new KeyStats();
for (String arg : opts.files) {
Path path = new Path(arg);
FileSystem fs = resolveFS(log, conf, path);
System.out
.println("Reading file: " + path.makeQualified(fs.getUri(), fs.getWorkingDirectory()));
printCryptoParams(path, fs);
CryptoService cs = CryptoFactoryLoader.getServiceForClient(CryptoEnvironment.Scope.TABLE,
siteConfig.getAllCryptoProperties());
CachableBuilder cb = new CachableBuilder().fsPath(fs, path).conf(conf).cryptoService(cs);
Reader iter = new RFile.Reader(cb);
MetricsGatherer<Map<String,ArrayList<VisibilityMetric>>> vmg = new VisMetricsGatherer();
if (opts.vis || opts.hash) {
iter.registerMetrics(vmg);
}
iter.printInfo(opts.printIndex);
System.out.println();
String propsPath = opts.getPropertiesPath();
String[] mainArgs =
propsPath == null ? new String[] {arg} : new String[] {"-props", propsPath, arg};
PrintBCInfo printBCInfo = new PrintBCInfo(mainArgs);
printBCInfo.setCryptoService(cs);
printBCInfo.printMetaBlockInfo();
Map<String,ArrayList<ByteSequence>> localityGroupCF = null;
if (opts.histogram || opts.dump || opts.vis || opts.hash || opts.keyStats || opts.fullKeys
|| !StringUtils.isEmpty(opts.formatterClazz)) {
localityGroupCF = iter.getLocalityGroupCF();
FileSKVIterator dataIter;
if (opts.useSample) {
dataIter = iter.getSample();
if (dataIter == null) {
System.out.println("ERROR : This rfile has no sample data");
return;
}
} else {
dataIter = iter;
}
if (opts.keyStats) {
FileSKVIterator indexIter = iter.getIndex();
while (indexIter.hasTop()) {
indexKeyStats.add(indexIter.getTopKey());
indexIter.next();
}
}
BiFunction<Key,Value,String> formatter = null;
if (opts.formatterClazz != null) {
final Class<? extends BiFunction<Key,Value,String>> formatterClass =
getFormatter(opts.formatterClazz);
formatter = formatterClass.getConstructor().newInstance();
} else if (opts.fullKeys) {
formatter = (key, value) -> key.toStringNoTruncate() + " -> " + value;
} else if (opts.dump) {
formatter = (key, value) -> key + " -> " + value;
}
for (String lgName : localityGroupCF.keySet()) {
LocalityGroupUtil.seek(dataIter, new Range(), lgName, localityGroupCF);
while (dataIter.hasTop()) {
Key key = dataIter.getTopKey();
Value value = dataIter.getTopValue();
if (formatter != null) {
System.out.println(formatter.apply(key, value));
if (System.out.checkError()) {
return;
}
}
if (opts.histogram) {
kvHistogram.add(key.getSize() + value.getSize());
}
if (opts.keyStats) {
dataKeyStats.add(key);
}
dataIter.next();
}
}
}
if (opts.printSummary) {
SummaryReader.print(iter, System.out);
}
iter.close();
if (opts.vis || opts.hash) {
System.out.println();
vmg.printMetrics(opts.hash, "Visibility", System.out);
}
if (opts.histogram) {
System.out.println();
kvHistogram.print("");
}
if (opts.keyStats) {
System.out.println();
System.out.println("Statistics for keys in data :");
dataKeyStats.print("\t");
System.out.println();
System.out.println("Statistics for keys in index :");
indexKeyStats.print("\t");
}
// If the output stream has closed, there is no reason to keep going.
if (System.out.checkError()) {
return;
}
}
}