[HUDI-700]Add unit test for FileSystemViewCommand (#1490)
This commit is contained in:
@@ -0,0 +1,56 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.cli;
|
||||
|
||||
/**
|
||||
* Fields of print table header.
|
||||
*/
|
||||
public class HoodieTableHeaderFields {
|
||||
public static final String HEADER_PARTITION = "Partition";
|
||||
public static final String HEADER_FILE_ID = "FileId";
|
||||
public static final String HEADER_BASE_INSTANT = "Base-Instant";
|
||||
|
||||
/**
|
||||
* Fields of data header.
|
||||
*/
|
||||
public static final String HEADER_DATA_FILE = "Data-File";
|
||||
public static final String HEADER_DATA_FILE_SIZE = HEADER_DATA_FILE + " Size";
|
||||
|
||||
/**
|
||||
* Fields of delta header.
|
||||
*/
|
||||
public static final String HEADER_DELTA_SIZE = "Delta Size";
|
||||
public static final String HEADER_DELTA_FILES = "Delta Files";
|
||||
public static final String HEADER_TOTAL_DELTA_SIZE = "Total " + HEADER_DELTA_SIZE;
|
||||
public static final String HEADER_TOTAL_DELTA_FILE_SIZE = "Total Delta File Size";
|
||||
public static final String HEADER_NUM_DELTA_FILES = "Num " + HEADER_DELTA_FILES;
|
||||
|
||||
/**
|
||||
* Fields of compaction scheduled header.
|
||||
*/
|
||||
private static final String COMPACTION_SCHEDULED_SUFFIX = " - compaction scheduled";
|
||||
private static final String COMPACTION_UNSCHEDULED_SUFFIX = " - compaction unscheduled";
|
||||
|
||||
public static final String HEADER_DELTA_SIZE_SCHEDULED = HEADER_DELTA_SIZE + COMPACTION_SCHEDULED_SUFFIX;
|
||||
public static final String HEADER_DELTA_SIZE_UNSCHEDULED = HEADER_DELTA_SIZE + COMPACTION_UNSCHEDULED_SUFFIX;
|
||||
public static final String HEADER_DELTA_BASE_SCHEDULED = "Delta To Base Ratio" + COMPACTION_SCHEDULED_SUFFIX;
|
||||
public static final String HEADER_DELTA_BASE_UNSCHEDULED = "Delta To Base Ratio" + COMPACTION_UNSCHEDULED_SUFFIX;
|
||||
public static final String HEADER_DELTA_FILES_SCHEDULED = "Delta Files" + COMPACTION_SCHEDULED_SUFFIX;
|
||||
public static final String HEADER_DELTA_FILES_UNSCHEDULED = "Delta Files" + COMPACTION_UNSCHEDULED_SUFFIX;
|
||||
}
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi.cli.commands;
|
||||
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
@@ -99,14 +100,18 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Total Delta File Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILE_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE, converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE);
|
||||
if (!baseFileOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta File Size")
|
||||
.addTableHeaderField("Delta Files");
|
||||
header = header.addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELTA_FILES)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILE_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES);
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
@@ -189,24 +194,28 @@ public class FileSystemViewCommand implements CommandMarker {
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put("Data-File Size", converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE, converterFunction);
|
||||
if (!baseFileOnly) {
|
||||
fieldNameToConverterMap.put("Total Delta Size", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction scheduled", converterFunction);
|
||||
fieldNameToConverterMap.put("Delta Size - compaction unscheduled", converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DELTA_SIZE_SCHEDULED, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DELTA_SIZE_UNSCHEDULED, converterFunction);
|
||||
}
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField("Partition").addTableHeaderField("FileId")
|
||||
.addTableHeaderField("Base-Instant").addTableHeaderField("Data-File").addTableHeaderField("Data-File Size");
|
||||
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE);
|
||||
|
||||
if (!baseFileOnly) {
|
||||
header = header.addTableHeaderField("Num Delta Files").addTableHeaderField("Total Delta Size")
|
||||
.addTableHeaderField("Delta Size - compaction scheduled")
|
||||
.addTableHeaderField("Delta Size - compaction unscheduled")
|
||||
.addTableHeaderField("Delta To Base Ratio - compaction scheduled")
|
||||
.addTableHeaderField("Delta To Base Ratio - compaction unscheduled")
|
||||
.addTableHeaderField("Delta Files - compaction scheduled")
|
||||
.addTableHeaderField("Delta Files - compaction unscheduled");
|
||||
header = header.addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELTA_FILES)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_SIZE_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_SIZE_UNSCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_BASE_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_BASE_UNSCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES_UNSCHEDULED);
|
||||
}
|
||||
return HoodiePrintHelper.print(header, fieldNameToConverterMap, sortByField, descending, limit, headerOnly, rows);
|
||||
}
|
||||
|
||||
@@ -0,0 +1,267 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.cli.commands;
|
||||
|
||||
import org.apache.hudi.cli.AbstractShellIntegrationTest;
|
||||
import org.apache.hudi.cli.HoodieCLI;
|
||||
import org.apache.hudi.cli.HoodiePrintHelper;
|
||||
import org.apache.hudi.cli.HoodieTableHeaderFields;
|
||||
import org.apache.hudi.cli.TableHeader;
|
||||
import org.apache.hudi.cli.common.HoodieTestCommitMetadataGenerator;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.util.NumericUtils;
|
||||
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.springframework.shell.core.CommandResult;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
/**
|
||||
* Test class for {@link FileSystemViewCommand}.
|
||||
*/
|
||||
public class TestFileSystemViewCommand extends AbstractShellIntegrationTest {
|
||||
private String partitionPath;
|
||||
private SyncableFileSystemView fsView;
|
||||
|
||||
@Before
|
||||
public void init() throws IOException {
|
||||
HoodieCLI.conf = jsc.hadoopConfiguration();
|
||||
|
||||
// Create table and connect
|
||||
String tableName = "test_table";
|
||||
String tablePath = basePath + File.separator + tableName;
|
||||
new TableCommand().createTable(
|
||||
tablePath, tableName,
|
||||
"COPY_ON_WRITE", "", 1, "org.apache.hudi.common.model.HoodieAvroPayload");
|
||||
|
||||
metaClient = HoodieCLI.getTableMetaClient();
|
||||
|
||||
partitionPath = HoodieTestCommitMetadataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
String fullPartitionPath = tablePath + "/" + partitionPath;
|
||||
new File(fullPartitionPath).mkdirs();
|
||||
|
||||
// Generate 2 commits
|
||||
String commitTime1 = "1";
|
||||
String commitTime2 = "2";
|
||||
|
||||
String fileId1 = UUID.randomUUID().toString();
|
||||
|
||||
// Write date files and log file
|
||||
String testWriteToken = "1-0-1";
|
||||
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime1, testWriteToken, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime1, 0, testWriteToken))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + "/" + FSUtils.makeDataFileName(commitTime2, testWriteToken, fileId1))
|
||||
.createNewFile();
|
||||
new File(fullPartitionPath + "/"
|
||||
+ FSUtils.makeLogFileName(fileId1, HoodieLogFile.DELTA_EXTENSION, commitTime2, 0, testWriteToken))
|
||||
.createNewFile();
|
||||
|
||||
// Write commit files
|
||||
new File(tablePath + "/.hoodie/" + commitTime1 + ".commit").createNewFile();
|
||||
new File(tablePath + "/.hoodie/" + commitTime2 + ".commit").createNewFile();
|
||||
|
||||
// Reload meta client and create fsView
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
fsView = new HoodieTableFileSystemView(metaClient, metaClient.getActiveTimeline(), true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for 'show fsview all'.
|
||||
*/
|
||||
@Test
|
||||
public void testShowCommits() {
|
||||
// Test default show fsview all
|
||||
CommandResult cr = getShell().executeCommand("show fsview all");
|
||||
assertTrue(cr.isSuccess());
|
||||
|
||||
// Get all file groups
|
||||
Stream<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath);
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
fileGroups.forEach(fg -> fg.getAllFileSlices().forEach(fs -> {
|
||||
int idx = 0;
|
||||
// For base file only Views, do not display any delta-file related columns
|
||||
Comparable[] row = new Comparable[8];
|
||||
row[idx++] = fg.getPartitionPath();
|
||||
row[idx++] = fg.getFileGroupId().getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : "";
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1;
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
row[idx++] = fs.getLogFiles().collect(Collectors.toList()).toString();
|
||||
rows.add(row);
|
||||
}));
|
||||
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILE_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE, converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELTA_FILES)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILE_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES);
|
||||
String expected = HoodiePrintHelper.print(header, fieldNameToConverterMap, "", false, -1, false, rows);
|
||||
assertEquals(expected, cr.getResult().toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for 'show fsview all' with specified values.
|
||||
*/
|
||||
@Test
|
||||
public void testShowCommitsWithSpecifiedValues() {
|
||||
// Test command with options, baseFileOnly and maxInstant is 2
|
||||
CommandResult cr = getShell().executeCommand("show fsview all --baseFileOnly true --maxInstant 2");
|
||||
assertTrue(cr.isSuccess());
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
Stream<HoodieFileGroup> fileGroups = fsView.getAllFileGroups(partitionPath);
|
||||
|
||||
// Only get instant 1, since maxInstant was specified 2
|
||||
fileGroups.forEach(fg -> fg.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals("1")).forEach(fs -> {
|
||||
int idx = 0;
|
||||
// For base file only Views, do not display any delta-file related columns.
|
||||
Comparable[] row = new Comparable[5];
|
||||
row[idx++] = fg.getPartitionPath();
|
||||
row[idx++] = fg.getFileGroupId().getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : "";
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1;
|
||||
rows.add(row);
|
||||
}));
|
||||
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_FILE_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE, converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE);
|
||||
|
||||
String expected = HoodiePrintHelper.print(header, fieldNameToConverterMap, "", false, -1, false, rows);
|
||||
assertEquals(expected, cr.getResult().toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for command 'show fsview latest'.
|
||||
*/
|
||||
@Test
|
||||
public void testShowLatestFileSlices() {
|
||||
// Test show with partition path '2016/03/15'
|
||||
CommandResult cr = getShell().executeCommand("show fsview latest --partitionPath " + partitionPath);
|
||||
assertTrue(cr.isSuccess());
|
||||
|
||||
Stream<FileSlice> fileSlice = fsView.getLatestFileSlices(partitionPath);
|
||||
|
||||
List<Comparable[]> rows = new ArrayList<>();
|
||||
fileSlice.forEach(fs -> {
|
||||
int idx = 0;
|
||||
// For base file only Views, do not display any delta-file related columns
|
||||
Comparable[] row = new Comparable[13];
|
||||
row[idx++] = partitionPath;
|
||||
row[idx++] = fs.getFileId();
|
||||
row[idx++] = fs.getBaseInstantTime();
|
||||
row[idx++] = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getPath() : "";
|
||||
|
||||
long dataFileSize = fs.getBaseFile().isPresent() ? fs.getBaseFile().get().getFileSize() : -1;
|
||||
row[idx++] = dataFileSize;
|
||||
|
||||
row[idx++] = fs.getLogFiles().count();
|
||||
row[idx++] = fs.getLogFiles().mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
long logFilesScheduledForCompactionTotalSize =
|
||||
fs.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
row[idx++] = logFilesScheduledForCompactionTotalSize;
|
||||
|
||||
long logFilesUnscheduledTotalSize =
|
||||
fs.getLogFiles().filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.mapToLong(HoodieLogFile::getFileSize).sum();
|
||||
row[idx++] = logFilesUnscheduledTotalSize;
|
||||
|
||||
double logSelectedForCompactionToBaseRatio =
|
||||
dataFileSize > 0 ? logFilesScheduledForCompactionTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logSelectedForCompactionToBaseRatio;
|
||||
double logUnscheduledToBaseRatio = dataFileSize > 0 ? logFilesUnscheduledTotalSize / (dataFileSize * 1.0) : -1;
|
||||
row[idx++] = logUnscheduledToBaseRatio;
|
||||
|
||||
row[idx++] = fs.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
row[idx++] = fs.getLogFiles().filter(lf -> !lf.getBaseCommitTime().equals(fs.getBaseInstantTime()))
|
||||
.collect(Collectors.toList()).toString();
|
||||
rows.add(row);
|
||||
});
|
||||
|
||||
Function<Object, String> converterFunction =
|
||||
entry -> NumericUtils.humanReadableByteCount((Double.parseDouble(entry.toString())));
|
||||
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_SIZE, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DELTA_SIZE_SCHEDULED, converterFunction);
|
||||
fieldNameToConverterMap.put(HoodieTableHeaderFields.HEADER_DELTA_SIZE_UNSCHEDULED, converterFunction);
|
||||
|
||||
TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_FILE_ID)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_BASE_INSTANT)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DATA_FILE_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_NUM_DELTA_FILES)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_TOTAL_DELTA_SIZE)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_SIZE_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_SIZE_UNSCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_BASE_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_BASE_UNSCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES_SCHEDULED)
|
||||
.addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES_UNSCHEDULED);
|
||||
String expected = HoodiePrintHelper.print(header, fieldNameToConverterMap, "", false, -1, false, rows);
|
||||
assertEquals(expected, cr.getResult().toString());
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user