1
0

[HUDI-699] Fix CompactionCommand and add unit test for CompactionCommand (#2325)

This commit is contained in:
hongdd
2021-04-08 15:35:33 +08:00
committed by GitHub
parent 18459d4045
commit ecdbd2517f
11 changed files with 725 additions and 84 deletions

View File

@@ -194,7 +194,7 @@ public class TestCommitsCommand extends AbstractShellIntegrationTest {
// archived 101 and 102 instants, remove 103 and 104 instant
data.remove("103");
data.remove("104");
String expected = generateExpectData(3, data);
String expected = generateExpectData(1, data);
expected = removeNonWordAndStripSpace(expected);
String got = removeNonWordAndStripSpace(cr.getResult().toString());
assertEquals(expected, got);

View File

@@ -0,0 +1,219 @@
/*
* 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.avro.model.HoodieCompactionPlan;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.HoodiePrintHelper;
import org.apache.hudi.cli.TableHeader;
import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.CompactionTestUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.springframework.shell.core.CommandResult;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;
/**
* Test Cases for {@link CompactionCommand}.
*/
public class TestCompactionCommand extends AbstractShellIntegrationTest {
private String tableName;
private String tablePath;
@BeforeEach
public void init() {
tableName = "test_table";
tablePath = basePath + tableName;
}
@Test
public void testVerifyTableType() throws IOException {
// create COW table.
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(),
"", TimelineLayoutVersion.VERSION_1, HoodieAvroPayload.class.getName());
// expect HoodieException for COPY_ON_WRITE table.
assertThrows(HoodieException.class,
() -> new CompactionCommand().compactionsAll(false, -1, "", false, false));
}
/**
* Test case for command 'compactions show all'.
*/
@Test
public void testCompactionsAll() throws IOException {
// create MOR table.
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.MERGE_ON_READ.name(),
"", TimelineLayoutVersion.VERSION_1, HoodieAvroPayload.class.getName());
CompactionTestUtils.setupAndValidateCompactionOperations(HoodieCLI.getTableMetaClient(), false, 3, 4, 3, 3);
HoodieCLI.getTableMetaClient().reloadActiveTimeline();
CommandResult cr = getShell().executeCommand("compactions show all");
System.out.println(cr.getResult().toString());
TableHeader header = new TableHeader().addTableHeaderField("Compaction Instant Time").addTableHeaderField("State")
.addTableHeaderField("Total FileIds to be Compacted");
Map<String, Integer> fileIds = new HashMap();
fileIds.put("001", 3);
fileIds.put("003", 4);
fileIds.put("005", 3);
fileIds.put("007", 3);
List<Comparable[]> rows = new ArrayList<>();
Arrays.asList("001", "003", "005", "007").stream().sorted(Comparator.reverseOrder()).forEach(instant -> {
rows.add(new Comparable[] {instant, "REQUESTED", fileIds.get(instant)});
});
String expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows);
assertEquals(expected, cr.getResult().toString());
}
/**
* Test case for command 'compaction show'.
*/
@Test
public void testCompactionShow() throws IOException {
// create MOR table.
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.MERGE_ON_READ.name(),
"", TimelineLayoutVersion.VERSION_1, HoodieAvroPayload.class.getName());
CompactionTestUtils.setupAndValidateCompactionOperations(HoodieCLI.getTableMetaClient(), false, 3, 4, 3, 3);
HoodieCLI.getTableMetaClient().reloadActiveTimeline();
CommandResult cr = getShell().executeCommand("compaction show --instant 001");
System.out.println(cr.getResult().toString());
}
private void generateCompactionInstances() throws IOException {
// create MOR table.
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.MERGE_ON_READ.name(),
"", TimelineLayoutVersion.VERSION_1, HoodieAvroPayload.class.getName());
CompactionTestUtils.setupAndValidateCompactionOperations(HoodieCLI.getTableMetaClient(), true, 1, 2, 3, 4);
HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().reloadActiveTimeline();
// Create six commits
Arrays.asList("001", "003", "005", "007").forEach(timestamp -> {
activeTimeline.transitionCompactionInflightToComplete(
new HoodieInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, timestamp), Option.empty());
});
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
}
private void generateArchive() throws IOException {
// Generate archive
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestCommitMetadataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
.forTable("test-trip-table").build();
// archive
metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient());
HoodieSparkTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
}
/**
* Test case for command 'compactions showarchived'.
*/
@Test
public void testCompactionsShowArchived() throws IOException {
generateCompactionInstances();
generateArchive();
CommandResult cr = getShell().executeCommand("compactions showarchived --startTs 001 --endTs 005");
// generate result
Map<String, Integer> fileMap = new HashMap<>();
fileMap.put("001", 1);
fileMap.put("003", 2);
fileMap.put("005", 3);
List<Comparable[]> rows = Arrays.asList("005", "003", "001").stream().map(i ->
new Comparable[] {i, HoodieInstant.State.COMPLETED, fileMap.get(i)}).collect(Collectors.toList());
Map<String, Function<Object, String>> fieldNameToConverterMap = new HashMap<>();
TableHeader header = new TableHeader().addTableHeaderField("Compaction Instant Time").addTableHeaderField("State")
.addTableHeaderField("Total FileIds to be Compacted");
String expected = HoodiePrintHelper.print(header, fieldNameToConverterMap, "", false, -1, false, rows);
expected = removeNonWordAndStripSpace(expected);
String got = removeNonWordAndStripSpace(cr.getResult().toString());
assertEquals(expected, got);
}
/**
* Test case for command 'compaction showarchived'.
*/
@Test
public void testCompactionShowArchived() throws IOException {
generateCompactionInstances();
String instance = "001";
// get compaction plan before compaction
HoodieCompactionPlan plan = TimelineMetadataUtils.deserializeCompactionPlan(
HoodieCLI.getTableMetaClient().reloadActiveTimeline().readCompactionPlanAsBytes(
HoodieTimeline.getCompactionRequestedInstant(instance)).get());
generateArchive();
CommandResult cr = getShell().executeCommand("compaction showarchived --instant " + instance);
// generate expected
String expected = new CompactionCommand().printCompaction(plan, "", false, -1, false);
expected = removeNonWordAndStripSpace(expected);
String got = removeNonWordAndStripSpace(cr.getResult().toString());
assertEquals(expected, got);
}
}

View File

@@ -0,0 +1,330 @@
/*
* 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.integ;
import org.apache.hudi.cli.HoodieCLI;
import org.apache.hudi.cli.commands.TableCommand;
import org.apache.hudi.cli.testutils.AbstractShellIntegrationTest;
import org.apache.hudi.client.CompactionAdminClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.TestCompactionAdminClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.testutils.CompactionTestUtils;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.springframework.shell.core.CommandResult;
import java.io.BufferedWriter;
import java.io.FileWriter;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.List;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertAll;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Integration test class for {@link org.apache.hudi.cli.commands.CompactionCommand}.
* <p/>
* A command use SparkLauncher need load jars under lib which generate during mvn package.
* Use integration test instead of unit test.
*/
public class ITTestCompactionCommand extends AbstractShellIntegrationTest {
private String tablePath;
private String tableName;
@BeforeEach
public void init() throws IOException {
tableName = "test_table_" + ITTestCompactionCommand.class.getName();
tablePath = Paths.get(basePath, tableName).toString();
HoodieCLI.conf = jsc.hadoopConfiguration();
// Create table and connect
new TableCommand().createTable(
tablePath, tableName, HoodieTableType.MERGE_ON_READ.name(),
"", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload");
metaClient.setBasePath(tablePath);
metaClient = HoodieTableMetaClient.reload(metaClient);
}
/**
* Test case for command 'compaction schedule'.
*/
@Test
public void testScheduleCompact() throws IOException {
// generate commits
generateCommits();
CommandResult cr = getShell().executeCommand(
String.format("compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1 --sparkMaster %s",
"local"));
assertAll("Command run failed",
() -> assertTrue(cr.isSuccess()),
() -> assertTrue(
cr.getResult().toString().startsWith("Attempted to schedule compaction for")));
// there is 1 requested compaction
HoodieActiveTimeline timeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
assertEquals(1, timeline.filterPendingCompactionTimeline().countInstants());
}
/**
* Test case for command 'compaction run'.
*/
@Test
public void testCompact() throws IOException {
// generate commits
generateCommits();
String instance = prepareScheduleCompaction();
String schemaPath = Paths.get(basePath, "compaction.schema").toString();
writeSchemaToTmpFile(schemaPath);
CommandResult cr2 = getShell().executeCommand(
String.format("compaction run --parallelism %s --schemaFilePath %s --sparkMaster %s",
2, schemaPath, "local"));
assertAll("Command run failed",
() -> assertTrue(cr2.isSuccess()),
() -> assertTrue(
cr2.getResult().toString().startsWith("Compaction successfully completed for")));
// assert compaction complete
assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload()
.filterCompletedInstants().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance),
"Pending compaction must be completed");
}
/**
* Test case for command 'compaction validate'.
*/
@Test
public void testValidateCompaction() throws IOException {
// generate commits
generateCommits();
String instance = prepareScheduleCompaction();
CommandResult cr = getShell().executeCommand(
String.format("compaction validate --instant %s --sparkMaster %s", instance, "local"));
assertAll("Command run failed",
() -> assertTrue(cr.isSuccess()),
() -> assertTrue(
// compaction requested should be valid
cr.getResult().toString().contains("COMPACTION PLAN VALID")));
}
/**
* This function mainly tests the workflow of 'compaction unschedule' command.
* The real test of {@link org.apache.hudi.client.CompactionAdminClient#unscheduleCompactionPlan}
* is {@link TestCompactionAdminClient#testUnscheduleCompactionPlan()}.
*/
@Test
public void testUnscheduleCompaction() throws Exception {
// generate commits
generateCommits();
String instance = prepareScheduleCompaction();
CommandResult cr = getShell().executeCommand(
String.format("compaction unschedule --instant %s --sparkMaster %s", instance, "local"));
// Always has no file
assertAll("Command run failed",
() -> assertTrue(cr.isSuccess()),
() -> assertEquals("No File renames needed to unschedule pending compaction. Operation successful.",
cr.getResult().toString()));
}
/**
* This function mainly tests the workflow of 'compaction unscheduleFileId' command.
* The real test of {@link org.apache.hudi.client.CompactionAdminClient#unscheduleCompactionFileId}
* is {@link TestCompactionAdminClient#testUnscheduleCompactionFileId}.
*/
@Test
public void testUnscheduleCompactFile() throws IOException {
int numEntriesPerInstant = 10;
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
CompactionOperation op = CompactionOperation.convertFromAvroRecordInstance(
CompactionUtils.getCompactionPlan(metaClient, "001").getOperations().stream().findFirst().get());
CommandResult cr = getShell().executeCommand(
String.format("compaction unscheduleFileId --fileId %s --partitionPath %s --sparkMaster %s",
op.getFileGroupId().getFileId(), op.getFileGroupId().getPartitionPath(), "local"));
assertAll("Command run failed",
() -> assertTrue(cr.isSuccess()),
() -> assertTrue(removeNonWordAndStripSpace(cr.getResult().toString()).contains("true")),
() -> assertFalse(removeNonWordAndStripSpace(cr.getResult().toString()).contains("false")));
}
/**
* This function mainly tests the workflow of 'compaction repair' command.
* The real test of {@link org.apache.hudi.client.CompactionAdminClient#repairCompaction}
* is {@link TestCompactionAdminClient#testRepairCompactionPlan}.
*/
@Test
public void testRepairCompaction() throws Exception {
int numEntriesPerInstant = 10;
String compactionInstant = "001";
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
metaClient.reloadActiveTimeline();
CompactionAdminClient client = new CompactionAdminClient(new HoodieSparkEngineContext(jsc), metaClient.getBasePath());
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1, Option.empty(), false);
renameFiles.forEach(lfPair -> {
try {
metaClient.getFs().rename(lfPair.getLeft().getPath(), lfPair.getRight().getPath());
} catch (IOException e) {
throw new HoodieIOException(e.getMessage(), e);
}
});
client.unscheduleCompactionPlan(compactionInstant, false, 1, false);
CommandResult cr = getShell().executeCommand(
String.format("compaction repair --instant %s --sparkMaster %s", compactionInstant, "local"));
// All Executes is succeeded, result contains true and has no false
// Expected:
// ║ File Id │ Source File Path │ Destination File Path │ Rename Executed? │ Rename Succeeded? │ Error ║
// ║ * │ * │ * │ true │ true │ ║
assertAll("Command run failed",
() -> assertTrue(cr.isSuccess()),
() -> assertTrue(removeNonWordAndStripSpace(cr.getResult().toString()).contains("true")),
() -> assertFalse(removeNonWordAndStripSpace(cr.getResult().toString()).contains("false")));
}
private String prepareScheduleCompaction() {
// generate requested compaction
CommandResult cr = getShell().executeCommand(
String.format("compaction schedule --hoodieConfigs hoodie.compact.inline.max.delta.commits=1 --sparkMaster %s",
"local"));
assertTrue(cr.isSuccess());
// get compaction instance
HoodieActiveTimeline timeline = HoodieCLI.getTableMetaClient().getActiveTimeline();
Option<String> instance =
timeline.filterPendingCompactionTimeline().firstInstant().map(HoodieInstant::getTimestamp);
assertTrue(instance.isPresent(), "Must have pending compaction.");
return instance.get();
}
private void writeSchemaToTmpFile(String schemaPath) throws IOException {
try (BufferedWriter out = new BufferedWriter(new FileWriter(schemaPath))) {
out.write(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
}
}
private void generateCommits() throws IOException {
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
// Create the write client to write some records in
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(tablePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2).forTable(tableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
SparkRDDWriteClient<HoodieAvroPayload> client = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), cfg);
List<HoodieRecord> records = insert(jsc, client, dataGen);
upsert(jsc, client, dataGen, records);
delete(jsc, client, records);
}
private List<HoodieRecord> insert(JavaSparkContext jsc, SparkRDDWriteClient<HoodieAvroPayload> client,
HoodieTestDataGenerator dataGen) throws IOException {
// inserts
String newCommitTime = "001";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 10);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
operateFunc(SparkRDDWriteClient::insert, client, writeRecords, newCommitTime);
return records;
}
private void upsert(JavaSparkContext jsc, SparkRDDWriteClient<HoodieAvroPayload> client,
HoodieTestDataGenerator dataGen, List<HoodieRecord> records)
throws IOException {
// updates
String newCommitTime = "002";
client.startCommitWithTime(newCommitTime);
List<HoodieRecord> toBeUpdated = dataGen.generateUpdates(newCommitTime, 2);
records.addAll(toBeUpdated);
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
operateFunc(SparkRDDWriteClient::upsert, client, writeRecords, newCommitTime);
}
private void delete(JavaSparkContext jsc, SparkRDDWriteClient<HoodieAvroPayload> client,
List<HoodieRecord> records) {
// Delete
String newCommitTime = "003";
client.startCommitWithTime(newCommitTime);
// just delete half of the records
int numToDelete = records.size() / 2;
List<HoodieKey> toBeDeleted = records.stream().map(HoodieRecord::getKey).limit(numToDelete).collect(Collectors.toList());
JavaRDD<HoodieKey> deleteRecords = jsc.parallelize(toBeDeleted, 1);
client.delete(deleteRecords, newCommitTime);
}
private JavaRDD<WriteStatus> operateFunc(
HoodieClientTestBase.Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
SparkRDDWriteClient<HoodieAvroPayload> client, JavaRDD<HoodieRecord> writeRecords, String commitTime)
throws IOException {
return writeFn.apply(client, writeRecords, commitTime);
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.cli.testutils;
import org.apache.hudi.common.model.HoodieTableType;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
@@ -37,4 +38,8 @@ public abstract class AbstractShellIntegrationTest extends AbstractShellBaseInte
public void teardown() throws Exception {
cleanupResources();
}
protected HoodieTableType getTableType() {
return HoodieTableType.MERGE_ON_READ;
}
}