[HUDI-1089] Refactor hudi-client to support multi-engine (#1827)
- This change breaks `hudi-client` into `hudi-client-common` and `hudi-spark-client` modules - Simple usages of Spark using jsc.parallelize() has been redone using EngineContext#map, EngineContext#flatMap etc - Code changes in the PR, break classes into `BaseXYZ` parent classes with no spark dependencies living in `hudi-client-common` - Classes on `hudi-spark-client` are named `SparkXYZ` extending the parent classes with all the Spark dependencies - To simplify/cleanup, HoodieIndex#fetchRecordLocation has been removed and its usages in tests replaced with alternatives Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -0,0 +1,320 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieRollbackException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test Cases for rollback of snapshots and commits.
|
||||
*/
|
||||
public class TestClientRollback extends HoodieClientTestBase {
|
||||
|
||||
/**
|
||||
* Test case for rollback-savepoint interaction.
|
||||
*/
|
||||
@Test
|
||||
public void testSavepointAndRollback() throws Exception {
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build()).build();
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS, basePath);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
List<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
client.savepoint("hoodie-unit-test", "test");
|
||||
|
||||
/**
|
||||
* Write 3 (updates)
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
List<String> partitionPaths =
|
||||
FSUtils.getAllPartitionPaths(fs, cfg.getBasePath(), getConfig().shouldAssumeDatePartitioning());
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view1 = table.getBaseFileOnlyView();
|
||||
|
||||
List<HoodieBaseFile> dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 003 should be present");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> {
|
||||
return view1.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"));
|
||||
}).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 002 should be present");
|
||||
|
||||
/**
|
||||
* Write 4 (updates)
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view2 = table.getBaseFileOnlyView();
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view2.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 004 should be present");
|
||||
|
||||
// rolling back to a non existent savepoint must not succeed
|
||||
assertThrows(HoodieRollbackException.class, () -> {
|
||||
client.restoreToSavepoint("001");
|
||||
}, "Rolling back to non-existent savepoint should not be allowed");
|
||||
|
||||
// rollback to savepoint 002
|
||||
HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstants().findFirst().get();
|
||||
client.restoreToSavepoint(savepoint.getTimestamp());
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
final BaseFileOnlyView view3 = table.getBaseFileOnlyView();
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("002"))).collect(Collectors.toList());
|
||||
assertEquals(3, dataFiles.size(), "The data files for commit 002 be available");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("003"))).collect(Collectors.toList());
|
||||
assertEquals(0, dataFiles.size(), "The data files for commit 003 should be rolled back");
|
||||
|
||||
dataFiles = partitionPaths.stream().flatMap(s -> view3.getAllBaseFiles(s).filter(f -> f.getCommitTime().equals("004"))).collect(Collectors.toList());
|
||||
assertEquals(0, dataFiles.size(), "The data files for commit 004 should be rolled back");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test Cases for effects of rollbacking completed/inflight commits.
|
||||
*/
|
||||
@Test
|
||||
public void testRollbackCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
final String commitTime1 = "20160501010101";
|
||||
final String commitTime2 = "20160502020601";
|
||||
final String commitTime3 = "20160506030611";
|
||||
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id11");
|
||||
put(p2, "id12");
|
||||
put(p3, "id13");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id21");
|
||||
put(p2, "id22");
|
||||
put(p3, "id23");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id31");
|
||||
put(p2, "id32");
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
|
||||
|
||||
// Rollback commit 1 (this should fail, since commit2 is still around)
|
||||
assertThrows(HoodieRollbackException.class, () -> {
|
||||
client.rollback(commitTime1);
|
||||
}, "Should have thrown an exception ");
|
||||
|
||||
// Rollback commit3
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// simulate partial failure, where .inflight was not deleted, but data files were.
|
||||
testTable.addInflightCommit(commitTime3);
|
||||
client.rollback(commitTime3);
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// Rollback commit2
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(testTable.commitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// simulate partial failure, where only .commit => .inflight renaming succeeded, leaving a
|
||||
// .inflight commit and a bunch of data files around.
|
||||
testTable.addInflightCommit(commitTime2).withBaseFilesInPartitions(partitionAndFileId2);
|
||||
|
||||
client.rollback(commitTime2);
|
||||
assertFalse(testTable.commitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
|
||||
// Let's rollback commit1, Check results
|
||||
client.rollback(commitTime1);
|
||||
assertFalse(testTable.commitExists(commitTime1));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime1));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test auto-rollback of commits which are in flight.
|
||||
*/
|
||||
@Test
|
||||
public void testAutoRollbackInflightCommit() throws Exception {
|
||||
// Let's create some commit files and parquet files
|
||||
final String p1 = "2016/05/01";
|
||||
final String p2 = "2016/05/02";
|
||||
final String p3 = "2016/05/06";
|
||||
final String commitTime1 = "20160501010101";
|
||||
final String commitTime2 = "20160502020601";
|
||||
final String commitTime3 = "20160506030611";
|
||||
Map<String, String> partitionAndFileId1 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id11");
|
||||
put(p2, "id12");
|
||||
put(p3, "id13");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId2 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id21");
|
||||
put(p2, "id22");
|
||||
put(p3, "id23");
|
||||
}
|
||||
};
|
||||
Map<String, String> partitionAndFileId3 = new HashMap<String, String>() {
|
||||
{
|
||||
put(p1, "id31");
|
||||
put(p2, "id32");
|
||||
put(p3, "id33");
|
||||
}
|
||||
};
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit(commitTime1)
|
||||
.withBaseFilesInPartitions(partitionAndFileId1)
|
||||
.addInflightCommit(commitTime2)
|
||||
.withBaseFilesInPartitions(partitionAndFileId2)
|
||||
.addInflightCommit(commitTime3)
|
||||
.withBaseFilesInPartitions(partitionAndFileId3);
|
||||
|
||||
// Turn auto rollback off
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
|
||||
final String commitTime4 = "20160506030621";
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, false)) {
|
||||
client.startCommitWithTime(commitTime4);
|
||||
// Check results, nothing changed
|
||||
assertTrue(testTable.commitExists(commitTime1));
|
||||
assertTrue(testTable.inflightCommitExists(commitTime2));
|
||||
assertTrue(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
}
|
||||
|
||||
// Turn auto rollback on
|
||||
final String commitTime5 = "20160506030631";
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(config, true)) {
|
||||
client.startCommitWithTime(commitTime5);
|
||||
assertTrue(testTable.commitExists(commitTime1));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime2));
|
||||
assertFalse(testTable.inflightCommitExists(commitTime3));
|
||||
assertTrue(testTable.baseFilesExist(partitionAndFileId1, commitTime1));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId2, commitTime2));
|
||||
assertFalse(testTable.baseFilesExist(partitionAndFileId3, commitTime3));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,353 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
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.testutils.CompactionTestUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
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.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.action.compact.OperationResult;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.client.CompactionAdminClient.getRenamingActionsToAlignWithCompactionOperation;
|
||||
import static org.apache.hudi.client.CompactionAdminClient.renameLogFile;
|
||||
import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestCompactionAdminClient extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestCompactionAdminClient.class);
|
||||
|
||||
private HoodieTableMetaClient metaClient;
|
||||
private CompactionAdminClient client;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
metaClient = HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath, MERGE_ON_READ);
|
||||
client = new CompactionAdminClient(context, basePath);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnSchedulePlan(client, "006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnscheduleCompactionFileId() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
Map<String, CompactionOperation> instantsWithOp =
|
||||
Stream.of("001", "003", "005", "007").map(instant -> {
|
||||
try {
|
||||
return Pair.of(instant, CompactionUtils.getCompactionPlan(metaClient, instant));
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieException(ioe);
|
||||
}
|
||||
}).map(instantWithPlan -> instantWithPlan.getRight().getOperations().stream()
|
||||
.map(op -> Pair.of(instantWithPlan.getLeft(), CompactionOperation.convertFromAvroRecordInstance(op)))
|
||||
.findFirst().get()).collect(Collectors.toMap(Pair::getLeft, Pair::getRight));
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "000", "001", instantsWithOp.get("001"), 2);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "002", "003", instantsWithOp.get("003"), 2);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "004", "005", instantsWithOp.get("005"), 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateUnScheduleFileId(client, "006", "007", instantsWithOp.get("007"), 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRepairCompactionPlan() throws Exception {
|
||||
int numEntriesPerInstant = 10;
|
||||
CompactionTestUtils.setupAndValidateCompactionOperations(metaClient, false, numEntriesPerInstant,
|
||||
numEntriesPerInstant, numEntriesPerInstant, numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("000", "001", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are delta-commits after compaction instant
|
||||
validateRepair("002", "003", numEntriesPerInstant, 2 * numEntriesPerInstant);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("004", "005", numEntriesPerInstant, 0);
|
||||
// THere are no delta-commits after compaction instant
|
||||
validateRepair("006", "007", numEntriesPerInstant, 0);
|
||||
}
|
||||
|
||||
private void validateRepair(String ingestionInstant, String compactionInstant, int numEntriesPerInstant,
|
||||
int expNumRepairs) throws Exception {
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRepairs, true);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
List<ValidationOpResult> result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
if (expNumRepairs > 0) {
|
||||
assertTrue(result.stream().anyMatch(r -> !r.isSuccess()), "Expect some failures in validation");
|
||||
}
|
||||
// Now repair
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> undoFiles =
|
||||
result.stream().flatMap(r -> getRenamingActionsToAlignWithCompactionOperation(metaClient,
|
||||
compactionInstant, r.getOperation(), Option.empty()).stream()).map(rn -> {
|
||||
try {
|
||||
renameLogFile(metaClient, rn.getKey(), rn.getValue());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
return rn;
|
||||
}).collect(Collectors.toList());
|
||||
Map<String, String> renameFilesFromUndo = undoFiles.stream()
|
||||
.collect(Collectors.toMap(p -> p.getRight().getPath().toString(), x -> x.getLeft().getPath().toString()));
|
||||
Map<String, String> expRenameFiles = renameFiles.stream()
|
||||
.collect(Collectors.toMap(p -> p.getLeft().getPath().toString(), x -> x.getRight().getPath().toString()));
|
||||
if (expNumRepairs > 0) {
|
||||
assertFalse(renameFiles.isEmpty(), "Rename Files must be non-empty");
|
||||
} else {
|
||||
assertTrue(renameFiles.isEmpty(), "Rename Files must be empty");
|
||||
}
|
||||
expRenameFiles.forEach((key, value) -> LOG.info("Key :" + key + " renamed to " + value + " rolled back to "
|
||||
+ renameFilesFromUndo.get(key)));
|
||||
|
||||
assertEquals(expRenameFiles, renameFilesFromUndo, "Undo must completely rollback renames");
|
||||
// Now expect validation to succeed
|
||||
result = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
assertTrue(result.stream().allMatch(OperationResult::isSuccess), "Expect no failures in validation");
|
||||
assertEquals(expNumRepairs, undoFiles.size(), "Expected Num Repairs");
|
||||
}
|
||||
|
||||
/**
|
||||
* Enssure compaction plan is valid.
|
||||
*
|
||||
* @param compactionInstant Compaction Instant
|
||||
*/
|
||||
private void ensureValidCompactionPlan(String compactionInstant) throws Exception {
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
// Ensure compaction-plan is good to begin with
|
||||
List<ValidationOpResult> validationResults = client.validateCompactionPlan(metaClient, compactionInstant, 1);
|
||||
assertFalse(validationResults.stream().anyMatch(v -> !v.isSuccess()),
|
||||
"Some validations failed");
|
||||
}
|
||||
|
||||
private void validateRenameFiles(List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles, String ingestionInstant,
|
||||
String compactionInstant, HoodieTableFileSystemView fsView) {
|
||||
// Ensure new names of log-files are on expected lines
|
||||
Set<HoodieLogFile> uniqNewLogFiles = new HashSet<>();
|
||||
Set<HoodieLogFile> uniqOldLogFiles = new HashSet<>();
|
||||
|
||||
renameFiles.forEach(lfPair -> {
|
||||
assertFalse(uniqOldLogFiles.contains(lfPair.getKey()), "Old Log File Names do not collide");
|
||||
assertFalse(uniqNewLogFiles.contains(lfPair.getValue()), "New Log File Names do not collide");
|
||||
uniqOldLogFiles.add(lfPair.getKey());
|
||||
uniqNewLogFiles.add(lfPair.getValue());
|
||||
});
|
||||
|
||||
renameFiles.forEach(lfPair -> {
|
||||
HoodieLogFile oldLogFile = lfPair.getLeft();
|
||||
HoodieLogFile newLogFile = lfPair.getValue();
|
||||
assertEquals(ingestionInstant, newLogFile.getBaseCommitTime(), "Base Commit time is expected");
|
||||
assertEquals(compactionInstant, oldLogFile.getBaseCommitTime(), "Base Commit time is expected");
|
||||
assertEquals(oldLogFile.getFileId(), newLogFile.getFileId(), "File Id is expected");
|
||||
HoodieLogFile lastLogFileBeforeCompaction =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], ingestionInstant)
|
||||
.filter(fs -> fs.getFileId().equals(oldLogFile.getFileId())).map(fs -> fs.getLogFiles().findFirst().get())
|
||||
.findFirst().get();
|
||||
assertEquals(lastLogFileBeforeCompaction.getLogVersion() + oldLogFile.getLogVersion(),
|
||||
newLogFile.getLogVersion(), "Log Version expected");
|
||||
assertTrue(newLogFile.getLogVersion() > lastLogFileBeforeCompaction.getLogVersion(),
|
||||
"Log version does not collide");
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames) throws Exception {
|
||||
return validateUnSchedulePlan(client, ingestionInstant, compactionInstant, numEntriesPerInstant, expNumRenames,
|
||||
false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private List<Pair<HoodieLogFile, HoodieLogFile>> validateUnSchedulePlan(CompactionAdminClient client,
|
||||
String ingestionInstant, String compactionInstant, int numEntriesPerInstant, int expNumRenames,
|
||||
boolean skipUnSchedule) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles =
|
||||
client.getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, 1, Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant)).flatMap(FileSlice::getLogFiles)
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(expLogFilesToBeRenamed, gotLogFilesToBeRenamed,
|
||||
"Log files belonging to file-slices created because of compaction request must be renamed");
|
||||
|
||||
if (skipUnSchedule) {
|
||||
// Do the renaming only but do not touch the compaction plan - Needed for repair tests
|
||||
renameFiles.forEach(lfPair -> {
|
||||
try {
|
||||
renameLogFile(metaClient, lfPair.getLeft(), lfPair.getRight());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException(e.getMessage(), e);
|
||||
}
|
||||
});
|
||||
} else {
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
}
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
|
||||
client.unscheduleCompactionPlan(compactionInstant, false, 1, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.forEach(fs -> {
|
||||
assertFalse(fs.getBaseFile().isPresent(), "No Data file must be present");
|
||||
assertEquals(0, fs.getLogFiles().count(), "No Log Files");
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
assertEquals(fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming,
|
||||
"Each File Id has same number of log-files");
|
||||
assertEquals(numEntriesPerInstant, fileIdToCountsAfterRenaming.size(), "Not Empty");
|
||||
assertEquals(expNumRenames, renameFiles.size(), "Expected number of renames");
|
||||
return renameFiles;
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate Unschedule operations.
|
||||
*/
|
||||
private void validateUnScheduleFileId(CompactionAdminClient client, String ingestionInstant, String compactionInstant,
|
||||
CompactionOperation op, int expNumRenames) throws Exception {
|
||||
|
||||
ensureValidCompactionPlan(compactionInstant);
|
||||
|
||||
// Check suggested rename operations
|
||||
List<Pair<HoodieLogFile, HoodieLogFile>> renameFiles = client
|
||||
.getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op, Option.empty(), false);
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
|
||||
// Log files belonging to file-slices created because of compaction request must be renamed
|
||||
|
||||
Set<HoodieLogFile> gotLogFilesToBeRenamed = renameFiles.stream().map(Pair::getLeft).collect(Collectors.toSet());
|
||||
final HoodieTableFileSystemView fsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
Set<HoodieLogFile> expLogFilesToBeRenamed = fsView.getLatestFileSlices(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0])
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId())).flatMap(FileSlice::getLogFiles)
|
||||
.collect(Collectors.toSet());
|
||||
assertEquals(expLogFilesToBeRenamed, gotLogFilesToBeRenamed,
|
||||
"Log files belonging to file-slices created because of compaction request must be renamed");
|
||||
validateRenameFiles(renameFiles, ingestionInstant, compactionInstant, fsView);
|
||||
|
||||
Map<String, Long> fileIdToCountsBeforeRenaming =
|
||||
fsView.getLatestMergedFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
// Call the main unschedule API
|
||||
client.unscheduleCompactionFileId(op.getFileGroupId(), false, false);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(metaClient.getHadoopConf(), basePath, true);
|
||||
final HoodieTableFileSystemView newFsView =
|
||||
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
|
||||
// Expect all file-slice whose base-commit is same as compaction commit to contain no new Log files
|
||||
newFsView.getLatestFileSlicesBeforeOrOn(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0], compactionInstant, true)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(compactionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.forEach(fs -> {
|
||||
assertFalse(fs.getBaseFile().isPresent(), "No Data file must be present");
|
||||
assertEquals(0, fs.getLogFiles().count(), "No Log Files");
|
||||
});
|
||||
|
||||
// Ensure same number of log-files before and after renaming per fileId
|
||||
Map<String, Long> fileIdToCountsAfterRenaming =
|
||||
newFsView.getAllFileGroups(HoodieTestUtils.DEFAULT_PARTITION_PATHS[0]).flatMap(HoodieFileGroup::getAllFileSlices)
|
||||
.filter(fs -> fs.getBaseInstantTime().equals(ingestionInstant))
|
||||
.filter(fs -> fs.getFileId().equals(op.getFileId()))
|
||||
.map(fs -> Pair.of(fs.getFileId(), fs.getLogFiles().count()))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
|
||||
assertEquals(fileIdToCountsBeforeRenaming, fileIdToCountsAfterRenaming,
|
||||
"Each File Id has same number of log-files");
|
||||
assertEquals(1, fileIdToCountsAfterRenaming.size(), "Not Empty");
|
||||
assertEquals(expNumRenames, renameFiles.size(), "Expected number of renames");
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieInternalWriteStatus}.
|
||||
*/
|
||||
public class TestHoodieInternalWriteStatus {
|
||||
|
||||
@Test
|
||||
public void testFailureFraction() {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String partitionPath = UUID.randomUUID().toString();
|
||||
status.setFileId(fileId);
|
||||
status.setPartitionPath(partitionPath);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markFailure(UUID.randomUUID().toString(), t);
|
||||
}
|
||||
// verification
|
||||
assertEquals(fileId, status.getFileId());
|
||||
assertEquals(partitionPath, status.getPartitionPath());
|
||||
assertTrue(status.getFailedRecordKeys().size() > 0);
|
||||
assertTrue(status.getFailedRecordKeys().size() < 150); // 150 instead of 100, to prevent flaky test
|
||||
assertTrue(status.hasErrors());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessRecordTracking() {
|
||||
boolean[] vals = {true, false};
|
||||
for (boolean trackSuccess : vals) {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(trackSuccess, 1.0);
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
status.setFileId(fileId);
|
||||
String partitionPath = UUID.randomUUID().toString();
|
||||
status.setPartitionPath(partitionPath);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markSuccess(UUID.randomUUID().toString());
|
||||
status.markFailure(UUID.randomUUID().toString(), t);
|
||||
}
|
||||
// verification
|
||||
assertEquals(fileId, status.getFileId());
|
||||
assertEquals(partitionPath, status.getPartitionPath());
|
||||
assertEquals(1000, status.getFailedRecordKeys().size());
|
||||
assertTrue(status.hasErrors());
|
||||
if (trackSuccess) {
|
||||
assertEquals(1000, status.getSuccessRecordKeys().size());
|
||||
} else {
|
||||
assertTrue(status.getSuccessRecordKeys().isEmpty());
|
||||
}
|
||||
assertEquals(2000, status.getTotalRecords());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGlobalError() {
|
||||
HoodieInternalWriteStatus status = new HoodieInternalWriteStatus(true, 0.1);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
status.setGlobalError(t);
|
||||
assertEquals(t, status.getGlobalError());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,235 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.AnalysisException;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
/**
|
||||
* Test-cases for covering HoodieReadClient APIs
|
||||
*/
|
||||
public class TestHoodieReadClient extends HoodieClientTestBase {
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insert.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsert() throws Exception {
|
||||
testReadFilterExist(getConfig(), SparkRDDWriteClient::insert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.insertPrepped.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfig(), SparkRDDWriteClient::insertPreppedRecords);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsert.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsert() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test ReadFilter API after writing new records using HoodieWriteClient.bulkInsertPrepped.
|
||||
*/
|
||||
@Test
|
||||
public void testReadFilterExistAfterBulkInsertPrepped() throws Exception {
|
||||
testReadFilterExist(getConfigBuilder().withBulkInsertParallelism(1).build(),
|
||||
(writeClient, recordRDD, instantTime) -> {
|
||||
return writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty());
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReadROViewFailsWithoutSqlContext() {
|
||||
HoodieReadClient readClient = new HoodieReadClient(context, getConfig());
|
||||
JavaRDD<HoodieKey> recordsRDD = jsc.parallelize(new ArrayList<>(), 1);
|
||||
assertThrows(IllegalStateException.class, () -> {
|
||||
readClient.readROView(recordsRDD, 1);
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to write new records using one of HoodieWriteClient's write API and use ReadClient to test filterExists()
|
||||
* API works correctly.
|
||||
*
|
||||
* @param config Hoodie Write Config
|
||||
* @param writeFn Write Function for writing records
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testReadFilterExist(HoodieWriteConfig config,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn) throws Exception {
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(config.getBasePath());
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<HoodieRecord> filteredRDD = readClient.filterExists(recordsRDD);
|
||||
|
||||
// Should not find any files
|
||||
assertEquals(100, filteredRDD.collect().size());
|
||||
|
||||
JavaRDD<HoodieRecord> smallRecordsRDD = jsc.parallelize(records.subList(0, 75), 1);
|
||||
// We create three parquet file, each having one record. (3 different partitions)
|
||||
List<WriteStatus> statuses = writeFn.apply(writeClient, smallRecordsRDD, newCommitTime).collect();
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
HoodieReadClient anotherReadClient = getHoodieReadClient(config.getBasePath());
|
||||
filteredRDD = anotherReadClient.filterExists(recordsRDD);
|
||||
List<HoodieRecord> result = filteredRDD.collect();
|
||||
// Check results
|
||||
assertEquals(25, result.size());
|
||||
|
||||
// check path exists for written keys
|
||||
JavaPairRDD<HoodieKey, Option<String>> keyToPathPair =
|
||||
anotherReadClient.checkExists(recordsRDD.map(HoodieRecord::getKey));
|
||||
JavaRDD<HoodieKey> keysWithPaths = keyToPathPair.filter(keyPath -> keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
assertEquals(75, keysWithPaths.count());
|
||||
|
||||
// verify rows match inserted records
|
||||
Dataset<Row> rows = anotherReadClient.readROView(keysWithPaths, 1);
|
||||
assertEquals(75, rows.count());
|
||||
|
||||
JavaRDD<HoodieKey> keysWithoutPaths = keyToPathPair.filter(keyPath -> !keyPath._2.isPresent())
|
||||
.map(keyPath -> keyPath._1);
|
||||
|
||||
assertThrows(AnalysisException.class, () -> {
|
||||
anotherReadClient.readROView(keysWithoutPaths, 1);
|
||||
});
|
||||
|
||||
// Actual tests of getPendingCompactions method are in TestAsyncCompaction
|
||||
// This is just testing empty list
|
||||
assertEquals(0, anotherReadClient.getPendingCompactions().size());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insert().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsert() throws Exception {
|
||||
testTagLocation(getConfig(), SparkRDDWriteClient::insert, SparkRDDWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after insertPrepped().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterInsertPrepped() throws Exception {
|
||||
testTagLocation(getConfig(), SparkRDDWriteClient::insertPreppedRecords, SparkRDDWriteClient::upsertPreppedRecords,
|
||||
true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulk-insert().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsert() throws Exception {
|
||||
testTagLocation(getConfigBuilder().withBulkInsertParallelism(1).build(), SparkRDDWriteClient::bulkInsert,
|
||||
SparkRDDWriteClient::upsert, false);
|
||||
}
|
||||
|
||||
/**
|
||||
* Test tagLocation API after bulkInsertPrepped().
|
||||
*/
|
||||
@Test
|
||||
public void testTagLocationAfterBulkInsertPrepped() throws Exception {
|
||||
testTagLocation(
|
||||
getConfigBuilder().withBulkInsertParallelism(1).build(), (writeClient, recordRDD, instantTime) -> writeClient
|
||||
.bulkInsertPreppedRecords(recordRDD, instantTime, Option.empty()),
|
||||
SparkRDDWriteClient::upsertPreppedRecords, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper method to test tagLocation after using different HoodieWriteClient write APIS.
|
||||
*
|
||||
* @param hoodieWriteConfig Write Config
|
||||
* @param insertFn Hoodie Write Client first Insert API
|
||||
* @param updateFn Hoodie Write Client upsert API
|
||||
* @param isPrepped isPrepped flag.
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
private void testTagLocation(HoodieWriteConfig hoodieWriteConfig,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> insertFn,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> updateFn, boolean isPrepped)
|
||||
throws Exception {
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig);) {
|
||||
// Write 1 (only inserts)
|
||||
String newCommitTime = "001";
|
||||
String initCommitTime = "000";
|
||||
int numRecords = 200;
|
||||
JavaRDD<WriteStatus> result = insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime,
|
||||
numRecords, insertFn, isPrepped, true, numRecords);
|
||||
// Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly
|
||||
// since they have been modified in the DAG
|
||||
JavaRDD<HoodieRecord> recordRDD =
|
||||
jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList()));
|
||||
// Should have 100 records in table (check using Index), all in locations marked at commit
|
||||
HoodieReadClient readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
List<HoodieRecord> taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
|
||||
// Write 2 (updates)
|
||||
String prevCommitTime = newCommitTime;
|
||||
newCommitTime = "004";
|
||||
numRecords = 100;
|
||||
String commitTimeBetweenPrevAndNew = "002";
|
||||
result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime,
|
||||
Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, updateFn, isPrepped, true,
|
||||
numRecords, 200, 2);
|
||||
recordRDD =
|
||||
jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecords).flatMap(Collection::stream)
|
||||
.map(record -> new HoodieRecord(record.getKey(), null)).collect(Collectors.toList()));
|
||||
// Index should be able to locate all updates in correct locations.
|
||||
readClient = getHoodieReadClient(hoodieWriteConfig.getBasePath());
|
||||
taggedRecords = readClient.tagLocation(recordRDD).collect();
|
||||
checkTaggedRecords(taggedRecords, newCommitTime);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,122 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
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.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestMultiFS extends HoodieClientTestHarness {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestMultiFS.class);
|
||||
private String tablePath = "file:///tmp/hoodie/sample-table";
|
||||
protected String tableName = "hoodie_rt";
|
||||
private String tableType = HoodieTableType.COPY_ON_WRITE.name();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initDFS();
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable(tableName)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void readLocalWriteHDFS() throws Exception {
|
||||
// Initialize table and filesystem
|
||||
HoodieTableMetaClient.initTableType(hadoopConf, dfsBasePath, HoodieTableType.valueOf(tableType),
|
||||
tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
// Create write client to write some records in
|
||||
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
|
||||
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
|
||||
|
||||
try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
|
||||
SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {
|
||||
|
||||
// Write generated data to hdfs (only inserts)
|
||||
String readCommitTime = hdfsWriteClient.startCommit();
|
||||
LOG.info("Starting commit " + readCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(readCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
hdfsWriteClient.upsert(writeRecords, readCommitTime);
|
||||
|
||||
// Read from hdfs
|
||||
FileSystem fs = FSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), dfsBasePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime);
|
||||
assertEquals(readRecords.count(), records.size(), "Should contain 100 records");
|
||||
|
||||
// Write to local
|
||||
HoodieTableMetaClient.initTableType(hadoopConf, tablePath, HoodieTableType.valueOf(tableType),
|
||||
tableName, HoodieAvroPayload.class.getName());
|
||||
|
||||
String writeCommitTime = localWriteClient.startCommit();
|
||||
LOG.info("Starting write commit " + writeCommitTime);
|
||||
List<HoodieRecord> localRecords = dataGen.generateInserts(writeCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> localWriteRecords = jsc.parallelize(localRecords, 1);
|
||||
LOG.info("Writing to path: " + tablePath);
|
||||
localWriteClient.upsert(localWriteRecords, writeCommitTime);
|
||||
|
||||
LOG.info("Reading from path: " + tablePath);
|
||||
fs = FSUtils.getFs(tablePath, HoodieTestUtils.getDefaultHadoopConf());
|
||||
metaClient = new HoodieTableMetaClient(fs.getConf(), tablePath);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
Dataset<Row> localReadRecords =
|
||||
HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime);
|
||||
assertEquals(localReadRecords.count(), localRecords.size(), "Should contain 100 records");
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,480 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
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.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_1;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.EXTRA_TYPE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.FARE_NESTED_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.MAP_TYPE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TIP_NESTED_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_PREFIX;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_SUFFIX;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestTableSchemaEvolution extends HoodieClientTestBase {
|
||||
|
||||
private final String initCommitTime = "000";
|
||||
private HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE;
|
||||
private HoodieTestDataGenerator dataGenEvolved = new HoodieTestDataGenerator();
|
||||
private HoodieTestDataGenerator dataGenDevolved = new HoodieTestDataGenerator();
|
||||
|
||||
public static final String EXTRA_FIELD_SCHEMA =
|
||||
"{\"name\": \"new_field\", \"type\": \"boolean\", \"default\": false},";
|
||||
|
||||
// TRIP_EXAMPLE_SCHEMA with a new_field added
|
||||
public static final String TRIP_EXAMPLE_SCHEMA_EVOLVED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
|
||||
+ FARE_NESTED_SCHEMA + TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
|
||||
// TRIP_EXAMPLE_SCHEMA with tip field removed
|
||||
public static final String TRIP_EXAMPLE_SCHEMA_DEVOLVED = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA
|
||||
+ FARE_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
|
||||
@Test
|
||||
public void testSchemaCompatibilityBasic() throws Exception {
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA),
|
||||
"Same schema is compatible");
|
||||
|
||||
String reorderedSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + TIP_NESTED_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ MAP_TYPE_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, reorderedSchema),
|
||||
"Reordered fields are compatible");
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(reorderedSchema, TRIP_EXAMPLE_SCHEMA),
|
||||
"Reordered fields are compatible");
|
||||
|
||||
String renamedSchema = TRIP_EXAMPLE_SCHEMA.replace("tip_history", "tip_future");
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedSchema),
|
||||
"Renamed fields are not compatible");
|
||||
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_DEVOLVED),
|
||||
"Deleted single field is not compatible");
|
||||
String deletedMultipleFieldSchema = TRIP_SCHEMA_PREFIX + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, deletedMultipleFieldSchema),
|
||||
"Deleted multiple fields are not compatible");
|
||||
|
||||
String renamedRecordSchema = TRIP_EXAMPLE_SCHEMA.replace("triprec", "triprec_renamed");
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, renamedRecordSchema),
|
||||
"Renamed record name is not compatible");
|
||||
|
||||
String swappedFieldSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA.replace("city_to_state", "fare")
|
||||
+ FARE_NESTED_SCHEMA.replace("fare", "city_to_state") + TIP_NESTED_SCHEMA + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, swappedFieldSchema),
|
||||
"Swapped fields are not compatible");
|
||||
|
||||
String typeChangeSchema = TRIP_SCHEMA_PREFIX + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ TIP_NESTED_SCHEMA.replace("string", "boolean") + TRIP_SCHEMA_SUFFIX;
|
||||
assertFalse(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, typeChangeSchema),
|
||||
"Field type change is not compatible");
|
||||
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, TRIP_EXAMPLE_SCHEMA_EVOLVED),
|
||||
"Added field with default is compatible (Evolved Schema)");
|
||||
|
||||
String multipleAddedFieldSchema = TRIP_SCHEMA_PREFIX + EXTRA_TYPE_SCHEMA + MAP_TYPE_SCHEMA + FARE_NESTED_SCHEMA
|
||||
+ TIP_NESTED_SCHEMA + EXTRA_FIELD_SCHEMA + EXTRA_FIELD_SCHEMA.replace("new_field", "new_new_field")
|
||||
+ TRIP_SCHEMA_SUFFIX;
|
||||
assertTrue(TableSchemaResolver.isSchemaCompatible(TRIP_EXAMPLE_SCHEMA, multipleAddedFieldSchema),
|
||||
"Multiple added fields with defauls are compatible");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMORTable() throws Exception {
|
||||
tableType = HoodieTableType.MERGE_ON_READ;
|
||||
|
||||
// Create the table
|
||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
|
||||
HoodieTableType.MERGE_ON_READ, metaClient.getTableConfig().getTableName(),
|
||||
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Initial inserts with TRIP_EXAMPLE_SCHEMA
|
||||
int numRecords = 10;
|
||||
insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
|
||||
numRecords, SparkRDDWriteClient::insert, false, false, numRecords);
|
||||
checkLatestDeltaCommit("001");
|
||||
|
||||
// Compact once so we can incrementally read later
|
||||
assertTrue(client.scheduleCompactionAtInstant("002", Option.empty()));
|
||||
client.compact("002");
|
||||
|
||||
// Updates with same schema is allowed
|
||||
final int numUpdateRecords = 5;
|
||||
updateBatch(hoodieWriteConfig, client, "003", "002", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
checkLatestDeltaCommit("003");
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Delete with same schema is allowed
|
||||
final int numDeleteRecords = 2;
|
||||
numRecords -= numDeleteRecords;
|
||||
deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords,
|
||||
SparkRDDWriteClient::delete, false, false, 0, 0);
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with evolved schema is not allowed
|
||||
HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false);
|
||||
final List<HoodieRecord> failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
try {
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a devolved schema and insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
|
||||
writeBatch(client, "005", "004", Option.empty(), "003", numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
|
||||
fail("Insert with devolved scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
client.rollback("005");
|
||||
}
|
||||
|
||||
// Update with devolved schema is also not allowed
|
||||
try {
|
||||
updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
fail("Update with devolved scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("004");
|
||||
checkReadRecords("000", numRecords);
|
||||
client.rollback("005");
|
||||
}
|
||||
|
||||
// Insert with an evolved scheme is allowed
|
||||
HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false);
|
||||
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a evolved schemaand insertBatch inserts records using the TRIP_EXMPLE_SCHEMA.
|
||||
final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("005", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "005", "004", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, 0, 0, 0);
|
||||
|
||||
// new commit
|
||||
checkLatestDeltaCommit("005");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Updates with evolved schema is allowed
|
||||
final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("006", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "006", "005", Option.empty(), initCommitTime,
|
||||
numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, 0, 0, 0);
|
||||
// new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Now even the original schema cannot be used for updates as it is devolved in relation to the
|
||||
// current schema of the dataset.
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
try {
|
||||
updateBatch(hoodieWriteConfig, client, "007", "006", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
fail("Update with original scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
client.rollback("007");
|
||||
}
|
||||
|
||||
// Now even the original schema cannot be used for inserts as it is devolved in relation to the
|
||||
// current schema of the dataset.
|
||||
try {
|
||||
// We are not using insertBatch directly here because insertion of these
|
||||
// records will fail and we dont want to keep these records within HoodieTestDataGenerator as we
|
||||
// will be testing updates later.
|
||||
failedRecords.clear();
|
||||
failedRecords.addAll(dataGen.generateInserts("007", numRecords));
|
||||
writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with original scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
checkLatestDeltaCommit("006");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
client.rollback("007");
|
||||
|
||||
// Remove the inserts from the in-memory state of HoodieTestDataGenerator
|
||||
// as these records were never inserted in the dataset. This is required so
|
||||
// that future calls to updateBatch or deleteBatch do not generate updates
|
||||
// or deletes for records which do not even exist.
|
||||
for (HoodieRecord record : failedRecords) {
|
||||
assertTrue(dataGen.deleteExistingKeyIfPresent(record.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
// Rollback to the original schema
|
||||
client.restoreToInstant("004");
|
||||
checkLatestDeltaCommit("004");
|
||||
|
||||
// Updates with original schema are now allowed
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
updateBatch(hoodieWriteConfig, client, "008", "004", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0);
|
||||
// new commit
|
||||
checkLatestDeltaCommit("008");
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Insert with original schema is allowed now
|
||||
insertBatch(hoodieWriteConfig, client, "009", "008", numRecords, SparkRDDWriteClient::insert,
|
||||
false, false, 0, 0, 0);
|
||||
checkLatestDeltaCommit("009");
|
||||
checkReadRecords("000", 3 * numRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyOnWriteTable() throws Exception {
|
||||
// Create the table
|
||||
HoodieTableMetaClient.initTableType(metaClient.getHadoopConf(), metaClient.getBasePath(),
|
||||
HoodieTableType.COPY_ON_WRITE, metaClient.getTableConfig().getTableName(),
|
||||
metaClient.getArchivePath(), metaClient.getTableConfig().getPayloadClass(), VERSION_1);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
|
||||
// Initial inserts with TRIP_EXAMPLE_SCHEMA
|
||||
int numRecords = 10;
|
||||
insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime,
|
||||
numRecords, SparkRDDWriteClient::insert, false, true, numRecords);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Updates with same schema is allowed
|
||||
final int numUpdateRecords = 5;
|
||||
updateBatch(hoodieWriteConfig, client, "002", "001", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, numRecords, 2);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Delete with same schema is allowed
|
||||
final int numDeleteRecords = 2;
|
||||
numRecords -= numDeleteRecords;
|
||||
deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords,
|
||||
SparkRDDWriteClient::delete, false, true, 0, numRecords);
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with devolved schema is not allowed
|
||||
HoodieWriteConfig hoodieDevolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
client = getHoodieWriteClient(hoodieDevolvedWriteConfig, false);
|
||||
final List<HoodieRecord> failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_DEVOLVED);
|
||||
try {
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a devolved schema.
|
||||
writeBatch(client, "004", "003", Option.empty(), "003", numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with devolved scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
client.rollback("004");
|
||||
}
|
||||
|
||||
// Update with devolved schema is not allowed
|
||||
try {
|
||||
updateBatch(hoodieDevolvedWriteConfig, client, "004", "003", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, 2 * numRecords, 5);
|
||||
fail("Update with devolved scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
client.rollback("004");
|
||||
}
|
||||
|
||||
// Insert with evolved scheme is allowed
|
||||
HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
client = getHoodieWriteClient(hoodieEvolvedWriteConfig, false);
|
||||
final List<HoodieRecord> evolvedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
// We cannot use insertBatch directly here because we want to insert records
|
||||
// with a evolved schema.
|
||||
writeBatch(client, "004", "003", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 2 * numRecords, 4);
|
||||
// new commit
|
||||
HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("004"));
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Updates with evolved schema is allowed
|
||||
final List<HoodieRecord> updateRecords = generateUpdatesWithSchema("005", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED);
|
||||
writeBatch(client, "005", "004", Option.empty(), initCommitTime,
|
||||
numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 2 * numRecords, 5);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Now even the original schema cannot be used for updates as it is devolved
|
||||
// in relation to the current schema of the dataset.
|
||||
client = getHoodieWriteClient(hoodieWriteConfig, false);
|
||||
try {
|
||||
updateBatch(hoodieWriteConfig, client, "006", "005", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, numRecords, 2);
|
||||
fail("Update with original scheme should fail");
|
||||
} catch (HoodieUpsertException ex) {
|
||||
// no new commit
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("005"));
|
||||
client.rollback("006");
|
||||
}
|
||||
|
||||
// Now even the original schema cannot be used for inserts as it is devolved
|
||||
// in relation to the current schema of the dataset.
|
||||
try {
|
||||
// We are not using insertBatch directly here because insertion of these
|
||||
// records will fail and we dont want to keep these records within
|
||||
// HoodieTestDataGenerator.
|
||||
failedRecords.clear();
|
||||
failedRecords.addAll(dataGen.generateInserts("006", numRecords));
|
||||
writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords,
|
||||
(String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords, 1);
|
||||
fail("Insert with original scheme should fail");
|
||||
} catch (HoodieInsertException ex) {
|
||||
// no new commit
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("005"));
|
||||
client.rollback("006");
|
||||
|
||||
// Remove the inserts from the in-memory state of HoodieTestDataGenerator
|
||||
// as these records were never inserted in the dataset. This is required so
|
||||
// that future calls to updateBatch or deleteBatch do not generate updates
|
||||
// or deletes for records which do not even exist.
|
||||
for (HoodieRecord record : failedRecords) {
|
||||
assertTrue(dataGen.deleteExistingKeyIfPresent(record.getKey()));
|
||||
}
|
||||
}
|
||||
|
||||
// Revert to the older commit and ensure that the original schema can now
|
||||
// be used for inserts and inserts.
|
||||
client.restoreToInstant("003");
|
||||
curTimeline = metaClient.reloadActiveTimeline().getCommitTimeline().filterCompletedInstants();
|
||||
assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("003"));
|
||||
checkReadRecords("000", numRecords);
|
||||
|
||||
// Insert with original schema is allowed now
|
||||
insertBatch(hoodieWriteConfig, client, "007", "003", numRecords, SparkRDDWriteClient::insert,
|
||||
false, true, numRecords, 2 * numRecords, 1);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
|
||||
// Update with original schema is allowed now
|
||||
updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(),
|
||||
initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true,
|
||||
numUpdateRecords, 2 * numRecords, 5);
|
||||
checkReadRecords("000", 2 * numRecords);
|
||||
}
|
||||
|
||||
private void checkReadRecords(String instantTime, int numExpectedRecords) throws IOException {
|
||||
if (tableType == HoodieTableType.COPY_ON_WRITE) {
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitTimeline();
|
||||
assertEquals(numExpectedRecords, HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, instantTime));
|
||||
} else {
|
||||
// TODO: This code fails to read records under the following conditions:
|
||||
// 1. No parquet files yet (i.e. no compaction done yet)
|
||||
// 2. Log file but no base file with the same FileID
|
||||
/*
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataAndLogFilesInPath(metaClient.getFs(), basePath);
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitsTimeline();
|
||||
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient, timeline, allFiles);
|
||||
List<String> dataFiles = fsView.getLatestBaseFiles().map(hf -> hf.getPath()).collect(Collectors.toList());
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
String absTableName = "hoodie." + metaClient.getTableConfig().getTableName();
|
||||
conf.set(absTableName + ".consume.mode", "INCREMENTAL");
|
||||
conf.set(absTableName + ".consume.start.timestamp", instantTime);
|
||||
conf.set(absTableName + ".consume.max.commits", "-1");
|
||||
List<GenericRecord> recordsRead = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(dataFiles, basePath, conf);
|
||||
assertEquals(recordsRead.size(), numExpectedRecords);
|
||||
*/
|
||||
}
|
||||
}
|
||||
|
||||
private void checkLatestDeltaCommit(String instantTime) {
|
||||
HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertTrue(timeline.lastInstant().get().getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION));
|
||||
assertTrue(timeline.lastInstant().get().getTimestamp().equals(instantTime));
|
||||
}
|
||||
|
||||
private List<HoodieRecord> generateInsertsWithSchema(String commitTime, int numRecords, String schemaStr) {
|
||||
HoodieTestDataGenerator gen = schemaStr.equals(TRIP_EXAMPLE_SCHEMA_EVOLVED) ? dataGenEvolved : dataGenDevolved;
|
||||
List<HoodieRecord> records = gen.generateInserts(commitTime, numRecords);
|
||||
return convertToSchema(records, schemaStr);
|
||||
}
|
||||
|
||||
private List<HoodieRecord> generateUpdatesWithSchema(String commitTime, int numRecords, String schemaStr) {
|
||||
HoodieTestDataGenerator gen = schemaStr.equals(TRIP_EXAMPLE_SCHEMA_EVOLVED) ? dataGenEvolved : dataGenDevolved;
|
||||
List<HoodieRecord> records = gen.generateUniqueUpdates(commitTime, numRecords);
|
||||
return convertToSchema(records, schemaStr);
|
||||
}
|
||||
|
||||
private List<HoodieRecord> convertToSchema(List<HoodieRecord> records, String schemaStr) {
|
||||
Schema newSchema = new Schema.Parser().parse(schemaStr);
|
||||
return records.stream().map(r -> {
|
||||
HoodieKey key = r.getKey();
|
||||
GenericRecord payload;
|
||||
try {
|
||||
payload = (GenericRecord)r.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA).get();
|
||||
GenericRecord newPayload = HoodieAvroUtils.rewriteRecordWithOnlyNewSchemaFields(payload, newSchema);
|
||||
return new HoodieRecord(key, new RawTripTestPayload(newPayload.toString(), key.getRecordKey(), key.getPartitionPath(), schemaStr));
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException("Conversion to new schema failed");
|
||||
}
|
||||
}).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getWriteConfig(String schema) {
|
||||
return getConfigBuilder(schema)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withAvroSchemaValidate(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return tableType;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,148 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestUpdateSchemaEvolution extends HoodieClientTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
HoodieTestUtils.init(HoodieTestUtils.getDefaultHadoopConf(), basePath);
|
||||
initSparkContexts("TestUpdateSchemaEvolution");
|
||||
initFileSystem();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSchemaEvolutionOnUpdate() throws Exception {
|
||||
// Create a bunch of records with a old version of schema
|
||||
final HoodieWriteConfig config = makeHoodieClientConfig("/exampleSchema.txt");
|
||||
final HoodieSparkTable table = HoodieSparkTable.create(config, context);
|
||||
|
||||
final List<WriteStatus> statuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
List<HoodieRecord> insertRecords = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
insertRecords
|
||||
.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
Map<String, HoodieRecord> insertRecordMap = insertRecords.stream()
|
||||
.collect(Collectors.toMap(r -> r.getRecordKey(), Function.identity()));
|
||||
HoodieCreateHandle createHandle =
|
||||
new HoodieCreateHandle(config, "100", table, rowChange1.getPartitionPath(), "f1-0", insertRecordMap, supplier);
|
||||
createHandle.write();
|
||||
return createHandle.close();
|
||||
}).collect();
|
||||
|
||||
final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" + HoodieTimeline.makeCommitFileName("100"));
|
||||
FSUtils.getFs(basePath, HoodieTestUtils.getDefaultHadoopConf()).create(commitFile);
|
||||
|
||||
// Now try an update with an evolved schema
|
||||
// Evolved schema does not have guarantee on preserving the original field ordering
|
||||
final HoodieWriteConfig config2 = makeHoodieClientConfig("/exampleEvolvedSchema.txt");
|
||||
final WriteStatus insertResult = statuses.get(0);
|
||||
String fileId = insertResult.getFileId();
|
||||
|
||||
final HoodieSparkTable table2 = HoodieSparkTable.create(config, context);
|
||||
assertEquals(1, jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
// New content with values for the newly added field
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12,\"added_field\":1}";
|
||||
List<HoodieRecord> updateRecords = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
record1.unseal();
|
||||
record1.setCurrentLocation(new HoodieRecordLocation("100", fileId));
|
||||
record1.seal();
|
||||
updateRecords.add(record1);
|
||||
|
||||
assertDoesNotThrow(() -> {
|
||||
HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config2, "101", table2,
|
||||
updateRecords.iterator(), record1.getPartitionPath(), fileId, supplier);
|
||||
Configuration conf = new Configuration();
|
||||
AvroReadSupport.setAvroReadSchema(conf, mergeHandle.getWriterSchemaWithMetafields());
|
||||
List<GenericRecord> oldRecords = ParquetUtils.readAvroRecords(conf,
|
||||
new Path(config2.getBasePath() + "/" + insertResult.getStat().getPath()));
|
||||
for (GenericRecord rec : oldRecords) {
|
||||
mergeHandle.write(rec);
|
||||
}
|
||||
mergeHandle.close();
|
||||
}, "UpdateFunction could not read records written with exampleSchema.txt using the "
|
||||
+ "exampleEvolvedSchema.txt");
|
||||
|
||||
return 1;
|
||||
}).collect().size());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig(String name) {
|
||||
Schema schema = getSchemaFromResource(getClass(), name);
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString()).build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.client;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
public class TestWriteStatus {
|
||||
@Test
|
||||
public void testFailureFraction() {
|
||||
WriteStatus status = new WriteStatus(true, 0.1);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markFailure(mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertTrue(status.getFailedRecords().size() > 0);
|
||||
assertTrue(status.getFailedRecords().size() < 150); // 150 instead of 100, to prevent flaky test
|
||||
assertTrue(status.hasErrors());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessRecordTracking() {
|
||||
WriteStatus status = new WriteStatus(false, 1.0);
|
||||
Throwable t = new Exception("some error in writing");
|
||||
for (int i = 0; i < 1000; i++) {
|
||||
status.markSuccess(mock(HoodieRecord.class), null);
|
||||
status.markFailure(mock(HoodieRecord.class), t, null);
|
||||
}
|
||||
assertEquals(1000, status.getFailedRecords().size());
|
||||
assertTrue(status.hasErrors());
|
||||
assertTrue(status.getWrittenRecords().isEmpty());
|
||||
assertEquals(2000, status.getTotalRecords());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,75 @@
|
||||
/*
|
||||
* 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.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.selector.BootstrapRegexModeSelector;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestBootstrapRegexModeSelector {
|
||||
|
||||
private HoodieWriteConfig getConfig(String regex, BootstrapMode selectedMode) {
|
||||
return HoodieWriteConfig.newBuilder().withPath("")
|
||||
.withBootstrapConfig(HoodieBootstrapConfig.newBuilder()
|
||||
.withBootstrapModeSelectorRegex(regex)
|
||||
.withBootstrapModeForRegexMatch(selectedMode).build())
|
||||
.forTable("test-trip-table").build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testModeSelector() {
|
||||
List<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||
String regex = "2020/05/1[0-9]";
|
||||
BootstrapRegexModeSelector regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex,
|
||||
BootstrapMode.FULL_RECORD));
|
||||
|
||||
Map<BootstrapMode, List<String>> result = regexModeSelector.select(input);
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/01"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/02"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/10"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/11"));
|
||||
assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size());
|
||||
assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size());
|
||||
|
||||
regexModeSelector = new BootstrapRegexModeSelector(getConfig(regex,
|
||||
BootstrapMode.METADATA_ONLY));
|
||||
result = regexModeSelector.select(input);
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/01"));
|
||||
assertTrue(result.get(BootstrapMode.FULL_RECORD).contains("2020/05/02"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/10"));
|
||||
assertTrue(result.get(BootstrapMode.METADATA_ONLY).contains("2020/05/11"));
|
||||
assertEquals(2, result.get(BootstrapMode.METADATA_ONLY).size());
|
||||
assertEquals(2, result.get(BootstrapMode.FULL_RECORD).size());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,66 @@
|
||||
/*
|
||||
* 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.client.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.selector.UniformBootstrapModeSelector;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestUniformBootstrapModeSelector {
|
||||
|
||||
@Test
|
||||
public void testFullBootstrapModeSelector() {
|
||||
|
||||
FullRecordBootstrapModeSelector modeSelector = new FullRecordBootstrapModeSelector(
|
||||
HoodieWriteConfig.newBuilder().withPath("").build());
|
||||
testModeSelector(modeSelector, BootstrapMode.FULL_RECORD);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMetadataOnlyBootstrapModeSelector() {
|
||||
MetadataOnlyBootstrapModeSelector modeSelector = new MetadataOnlyBootstrapModeSelector(
|
||||
HoodieWriteConfig.newBuilder().withPath("").build());
|
||||
testModeSelector(modeSelector, BootstrapMode.METADATA_ONLY);
|
||||
}
|
||||
|
||||
private void testModeSelector(UniformBootstrapModeSelector modeSelector, BootstrapMode mode) {
|
||||
List<String> partitionPaths = Arrays.asList("2020/05/01", "2020/05/02", "2020/05/10", "2020/05/11");
|
||||
List<Pair<String, List<HoodieFileStatus>>> input = partitionPaths.stream()
|
||||
.map(p -> Pair.<String, List<HoodieFileStatus>>of(p, new ArrayList<>())).collect(Collectors.toList());
|
||||
Map<BootstrapMode, List<String>> result = modeSelector.select(input);
|
||||
assertTrue(result.get(mode).contains("2020/05/01"));
|
||||
assertTrue(result.get(mode).contains("2020/05/02"));
|
||||
assertTrue(result.get(mode).contains("2020/05/10"));
|
||||
assertTrue(result.get(mode).contains("2020/05/11"));
|
||||
assertEquals(4, result.get(mode).size());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,239 @@
|
||||
/*
|
||||
* 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.client.model;
|
||||
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieInternalRow}.
|
||||
*/
|
||||
public class TestHoodieInternalRow {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
private static final int INTEGER_INDEX = 5;
|
||||
private static final int STRING_INDEX = 6;
|
||||
private static final int BOOLEAN_INDEX = 7;
|
||||
private static final int SHORT_INDEX = 8;
|
||||
private static final int BYTE_INDEX = 9;
|
||||
private static final int LONG_INDEX = 10;
|
||||
private static final int FLOAT_INDEX = 11;
|
||||
private static final int DOUBLE_INDEX = 12;
|
||||
private static final int DECIMAL_INDEX = 13;
|
||||
private static final int BINARY_INDEX = 14;
|
||||
private static final int STRUCT_INDEX = 15;
|
||||
// to do array and map
|
||||
private static final int ARRAY_INDEX = 16;
|
||||
private static final int MAP_INDEX = 17;
|
||||
|
||||
private List<Integer> nullIndices;
|
||||
|
||||
public TestHoodieInternalRow() {
|
||||
this.nullIndices = new ArrayList<>();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGet() {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdate() {
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
hoodieInternalRow.update(0, "commitTime_updated");
|
||||
hoodieInternalRow.update(1, "commitSeqNo_updated");
|
||||
hoodieInternalRow.update(2, "recordKey_updated");
|
||||
hoodieInternalRow.update(3, "partitionPath_updated");
|
||||
hoodieInternalRow.update(4, "fileName_updated");
|
||||
|
||||
values = getRandomValue(true);
|
||||
hoodieInternalRow.update(INTEGER_INDEX, values[INTEGER_INDEX]);
|
||||
hoodieInternalRow.update(BOOLEAN_INDEX, values[BOOLEAN_INDEX]);
|
||||
hoodieInternalRow.update(SHORT_INDEX, values[SHORT_INDEX]);
|
||||
hoodieInternalRow.update(BYTE_INDEX, values[BYTE_INDEX]);
|
||||
hoodieInternalRow.update(LONG_INDEX, values[LONG_INDEX]);
|
||||
hoodieInternalRow.update(FLOAT_INDEX, values[FLOAT_INDEX]);
|
||||
hoodieInternalRow.update(DOUBLE_INDEX, values[DOUBLE_INDEX]);
|
||||
//hoodieInternalRow.update(decimalIndex, values[decimalIndex]);
|
||||
hoodieInternalRow.update(BINARY_INDEX, values[BINARY_INDEX]);
|
||||
hoodieInternalRow.update(STRUCT_INDEX, values[STRUCT_INDEX]);
|
||||
hoodieInternalRow.update(STRING_INDEX, values[STRING_INDEX].toString());
|
||||
|
||||
assertValues(hoodieInternalRow, "commitTime_updated", "commitSeqNo_updated", "recordKey_updated", "partitionPath_updated",
|
||||
"fileName_updated", values, nullIndices);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testIsNullCheck() {
|
||||
|
||||
for (int i = 0; i < 16; i++) {
|
||||
Object[] values = getRandomValue(true);
|
||||
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
hoodieInternalRow.setNullAt(i);
|
||||
nullIndices.clear();
|
||||
nullIndices.add(i);
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
|
||||
// try setting multiple values as null
|
||||
// run it for 5 rounds
|
||||
for (int i = 0; i < 5; i++) {
|
||||
int numNullValues = 1 + RANDOM.nextInt(4);
|
||||
List<Integer> nullsSoFar = new ArrayList<>();
|
||||
while (nullsSoFar.size() < numNullValues) {
|
||||
int randomIndex = RANDOM.nextInt(16);
|
||||
if (!nullsSoFar.contains(randomIndex)) {
|
||||
nullsSoFar.add(randomIndex);
|
||||
}
|
||||
}
|
||||
|
||||
Object[] values = getRandomValue(true);
|
||||
InternalRow row = new GenericInternalRow(values);
|
||||
HoodieInternalRow hoodieInternalRow = new HoodieInternalRow("commitTime", "commitSeqNo", "recordKey", "partitionPath", "fileName", row);
|
||||
|
||||
nullIndices.clear();
|
||||
|
||||
for (Integer index : nullsSoFar) {
|
||||
hoodieInternalRow.setNullAt(index);
|
||||
nullIndices.add(index);
|
||||
}
|
||||
assertValues(hoodieInternalRow, "commitTime", "commitSeqNo", "recordKey", "partitionPath",
|
||||
"fileName", values, nullIndices);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetches a random Object[] of values for testing.
|
||||
*
|
||||
* @param withStructType true if structType need to be added as one of the elements in the Object[]
|
||||
* @return the random Object[] thus generated
|
||||
*/
|
||||
private Object[] getRandomValue(boolean withStructType) {
|
||||
Object[] values = new Object[16];
|
||||
values[INTEGER_INDEX] = RANDOM.nextInt();
|
||||
values[STRING_INDEX] = UUID.randomUUID().toString();
|
||||
values[BOOLEAN_INDEX] = RANDOM.nextBoolean();
|
||||
values[SHORT_INDEX] = (short) RANDOM.nextInt(2);
|
||||
byte[] bytes = new byte[1];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BYTE_INDEX] = bytes[0];
|
||||
values[LONG_INDEX] = RANDOM.nextLong();
|
||||
values[FLOAT_INDEX] = RANDOM.nextFloat();
|
||||
values[DOUBLE_INDEX] = RANDOM.nextDouble();
|
||||
// TODO fix decimal type.
|
||||
values[DECIMAL_INDEX] = RANDOM.nextFloat();
|
||||
bytes = new byte[20];
|
||||
RANDOM.nextBytes(bytes);
|
||||
values[BINARY_INDEX] = bytes;
|
||||
if (withStructType) {
|
||||
Object[] structField = getRandomValue(false);
|
||||
values[STRUCT_INDEX] = new GenericInternalRow(structField);
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
private void assertValues(HoodieInternalRow hoodieInternalRow, String commitTime, String commitSeqNo, String recordKey, String partitionPath, String filename, Object[] values,
|
||||
List<Integer> nullIndexes) {
|
||||
for (Integer index : nullIndexes) {
|
||||
assertTrue(hoodieInternalRow.isNullAt(index));
|
||||
}
|
||||
for (int i = 0; i < 16; i++) {
|
||||
if (!nullIndexes.contains(i)) {
|
||||
assertFalse(hoodieInternalRow.isNullAt(i));
|
||||
}
|
||||
}
|
||||
if (!nullIndexes.contains(0)) {
|
||||
assertEquals(commitTime, hoodieInternalRow.get(0, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(1)) {
|
||||
assertEquals(commitSeqNo, hoodieInternalRow.get(1, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(2)) {
|
||||
assertEquals(recordKey, hoodieInternalRow.get(2, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(3)) {
|
||||
assertEquals(partitionPath, hoodieInternalRow.get(3, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(4)) {
|
||||
assertEquals(filename, hoodieInternalRow.get(4, DataTypes.StringType).toString());
|
||||
}
|
||||
if (!nullIndexes.contains(INTEGER_INDEX)) {
|
||||
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.getInt(INTEGER_INDEX));
|
||||
assertEquals(values[INTEGER_INDEX], hoodieInternalRow.get(INTEGER_INDEX, DataTypes.IntegerType));
|
||||
}
|
||||
if (!nullIndexes.contains(STRING_INDEX)) {
|
||||
assertEquals(values[STRING_INDEX].toString(), hoodieInternalRow.get(STRING_INDEX, DataTypes.StringType));
|
||||
}
|
||||
if (!nullIndexes.contains(BOOLEAN_INDEX)) {
|
||||
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.getBoolean(BOOLEAN_INDEX));
|
||||
assertEquals(values[BOOLEAN_INDEX], hoodieInternalRow.get(BOOLEAN_INDEX, DataTypes.BooleanType));
|
||||
}
|
||||
if (!nullIndexes.contains(SHORT_INDEX)) {
|
||||
assertEquals(values[SHORT_INDEX], hoodieInternalRow.getShort(SHORT_INDEX));
|
||||
assertEquals(values[SHORT_INDEX], hoodieInternalRow.get(SHORT_INDEX, DataTypes.ShortType));
|
||||
}
|
||||
if (!nullIndexes.contains(BYTE_INDEX)) {
|
||||
assertEquals(values[BYTE_INDEX], hoodieInternalRow.getByte(BYTE_INDEX));
|
||||
assertEquals(values[BYTE_INDEX], hoodieInternalRow.get(BYTE_INDEX, DataTypes.ByteType));
|
||||
}
|
||||
if (!nullIndexes.contains(LONG_INDEX)) {
|
||||
assertEquals(values[LONG_INDEX], hoodieInternalRow.getLong(LONG_INDEX));
|
||||
assertEquals(values[LONG_INDEX], hoodieInternalRow.get(LONG_INDEX, DataTypes.LongType));
|
||||
}
|
||||
if (!nullIndexes.contains(FLOAT_INDEX)) {
|
||||
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.getFloat(FLOAT_INDEX));
|
||||
assertEquals(values[FLOAT_INDEX], hoodieInternalRow.get(FLOAT_INDEX, DataTypes.FloatType));
|
||||
}
|
||||
if (!nullIndexes.contains(DOUBLE_INDEX)) {
|
||||
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.getDouble(DOUBLE_INDEX));
|
||||
assertEquals(values[DOUBLE_INDEX], hoodieInternalRow.get(DOUBLE_INDEX, DataTypes.DoubleType));
|
||||
}
|
||||
if (!nullIndexes.contains(BINARY_INDEX)) {
|
||||
assertEquals(values[BINARY_INDEX], hoodieInternalRow.getBinary(BINARY_INDEX));
|
||||
assertEquals(values[BINARY_INDEX], hoodieInternalRow.get(BINARY_INDEX, DataTypes.BinaryType));
|
||||
}
|
||||
if (!nullIndexes.contains(STRUCT_INDEX)) {
|
||||
assertEquals(values[STRUCT_INDEX], hoodieInternalRow.getStruct(STRUCT_INDEX, 18));
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,332 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.DefaultSizeEstimator;
|
||||
import org.apache.hudi.common.util.FileIOUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.SizeEstimator;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueue;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.FunctionBasedQueueProducer;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.Timeout;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestBoundedInMemoryQueue extends HoodieClientTestHarness {
|
||||
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
initExecutorServiceWithFixedThreadPool(2);
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
// Test to ensure that we are reading all records from queue iterator in the same order
|
||||
// without any exceptions.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testRecordReading() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
// Produce
|
||||
Future<Boolean> resFuture = executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
|
||||
queue.close();
|
||||
return true;
|
||||
});
|
||||
final Iterator<HoodieRecord> originalRecordIterator = hoodieRecords.iterator();
|
||||
int recordsRead = 0;
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieRecord originalRecord = originalRecordIterator.next();
|
||||
final Option<IndexedRecord> originalInsertValue =
|
||||
originalRecord.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA);
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
|
||||
// Ensure that record ordering is guaranteed.
|
||||
assertEquals(originalRecord, payload.record);
|
||||
// cached insert value matches the expected insert value.
|
||||
assertEquals(originalInsertValue,
|
||||
payload.record.getData().getInsertValue(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
recordsRead++;
|
||||
}
|
||||
assertFalse(queue.iterator().hasNext() || originalRecordIterator.hasNext());
|
||||
// all the records should be read successfully.
|
||||
assertEquals(numRecords, recordsRead);
|
||||
// should not throw any exceptions.
|
||||
resFuture.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Test to ensure that we are reading all records from queue iterator when we have multiple producers.
|
||||
*/
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testCompositeProducerRecordReading() throws Exception {
|
||||
final int numRecords = 1000;
|
||||
final int numProducers = 40;
|
||||
final List<List<HoodieRecord>> recs = new ArrayList<>();
|
||||
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(FileIOUtils.KB, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Record Key to <Producer Index, Rec Index within a producer>
|
||||
Map<String, Tuple2<Integer, Integer>> keyToProducerAndIndexMap = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
List<HoodieRecord> pRecs = dataGen.generateInserts(instantTime, numRecords);
|
||||
int j = 0;
|
||||
for (HoodieRecord r : pRecs) {
|
||||
assertFalse(keyToProducerAndIndexMap.containsKey(r.getRecordKey()));
|
||||
keyToProducerAndIndexMap.put(r.getRecordKey(), new Tuple2<>(i, j));
|
||||
j++;
|
||||
}
|
||||
recs.add(pRecs);
|
||||
}
|
||||
|
||||
List<BoundedInMemoryQueueProducer<HoodieRecord>> producers = new ArrayList<>();
|
||||
for (int i = 0; i < recs.size(); i++) {
|
||||
final List<HoodieRecord> r = recs.get(i);
|
||||
// Alternate between pull and push based iterators
|
||||
if (i % 2 == 0) {
|
||||
producers.add(new IteratorBasedQueueProducer<>(r.iterator()));
|
||||
} else {
|
||||
producers.add(new FunctionBasedQueueProducer<>((buf) -> {
|
||||
Iterator<HoodieRecord> itr = r.iterator();
|
||||
while (itr.hasNext()) {
|
||||
try {
|
||||
buf.insertRecord(itr.next());
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
||||
final List<Future<Boolean>> futureList = producers.stream().map(producer -> {
|
||||
return executorService.submit(() -> {
|
||||
producer.produce(queue);
|
||||
return true;
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
|
||||
// Close queue
|
||||
Future<Boolean> closeFuture = executorService.submit(() -> {
|
||||
try {
|
||||
for (Future f : futureList) {
|
||||
f.get();
|
||||
}
|
||||
queue.close();
|
||||
} catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
return true;
|
||||
});
|
||||
|
||||
// Used to ensure that consumer sees the records generated by a single producer in FIFO order
|
||||
Map<Integer, Integer> lastSeenMap =
|
||||
IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> -1));
|
||||
Map<Integer, Integer> countMap =
|
||||
IntStream.range(0, numProducers).boxed().collect(Collectors.toMap(Function.identity(), x -> 0));
|
||||
|
||||
// Read recs and ensure we have covered all producer recs.
|
||||
while (queue.iterator().hasNext()) {
|
||||
final HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload = queue.iterator().next();
|
||||
final HoodieRecord rec = payload.record;
|
||||
Tuple2<Integer, Integer> producerPos = keyToProducerAndIndexMap.get(rec.getRecordKey());
|
||||
Integer lastSeenPos = lastSeenMap.get(producerPos._1());
|
||||
countMap.put(producerPos._1(), countMap.get(producerPos._1()) + 1);
|
||||
lastSeenMap.put(producerPos._1(), lastSeenPos + 1);
|
||||
// Ensure we are seeing the next record generated
|
||||
assertEquals(lastSeenPos + 1, producerPos._2().intValue());
|
||||
}
|
||||
|
||||
for (int i = 0; i < numProducers; i++) {
|
||||
// Ensure we have seen all the records for each producers
|
||||
assertEquals(Integer.valueOf(numRecords), countMap.get(i));
|
||||
}
|
||||
|
||||
// Ensure Close future is done
|
||||
closeFuture.get();
|
||||
}
|
||||
|
||||
// Test to ensure that record queueing is throttled when we hit memory limit.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testMemoryLimitForBuffering() throws Exception {
|
||||
final int numRecords = 128;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
// maximum number of records to keep in memory.
|
||||
final int recordLimit = 5;
|
||||
final SizeEstimator<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
|
||||
final long objSize = sizeEstimator.sizeEstimate(payload);
|
||||
final long memoryLimitInBytes = recordLimit * objSize;
|
||||
final BoundedInMemoryQueue<HoodieRecord, HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>> queue =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue);
|
||||
return true;
|
||||
});
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
assertEquals(0, queue.rateLimiter.availablePermits());
|
||||
assertEquals(recordLimit, queue.currentRateLimit);
|
||||
assertEquals(recordLimit, queue.size());
|
||||
assertEquals(recordLimit - 1, queue.samplingRecordCounter.get());
|
||||
|
||||
// try to read 2 records.
|
||||
assertEquals(hoodieRecords.get(0), queue.iterator().next().record);
|
||||
assertEquals(hoodieRecords.get(1), queue.iterator().next().record);
|
||||
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
// No change is expected in rate limit or number of queued records. We only expect
|
||||
// queueing thread to read
|
||||
// 2 more records into the queue.
|
||||
assertEquals(0, queue.rateLimiter.availablePermits());
|
||||
assertEquals(recordLimit, queue.currentRateLimit);
|
||||
assertEquals(recordLimit, queue.size());
|
||||
assertEquals(recordLimit - 1 + 2, queue.samplingRecordCounter.get());
|
||||
}
|
||||
|
||||
// Test to ensure that exception in either queueing thread or BufferedIterator-reader thread
|
||||
// is propagated to
|
||||
// another thread.
|
||||
@SuppressWarnings("unchecked")
|
||||
@Test
|
||||
@Timeout(value = 60)
|
||||
public void testException() throws Exception {
|
||||
final int numRecords = 256;
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, numRecords);
|
||||
final SizeEstimator<Tuple2<HoodieRecord, Option<IndexedRecord>>> sizeEstimator = new DefaultSizeEstimator<>();
|
||||
// queue memory limit
|
||||
HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> payload =
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA).apply(hoodieRecords.get(0));
|
||||
final long objSize = sizeEstimator.sizeEstimate(new Tuple2<>(payload.record, payload.insertValue));
|
||||
final long memoryLimitInBytes = 4 * objSize;
|
||||
|
||||
// first let us throw exception from queueIterator reader and test that queueing thread
|
||||
// stops and throws
|
||||
// correct exception back.
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue1 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
Future<Boolean> resFuture = executorService.submit(() -> {
|
||||
new IteratorBasedQueueProducer<>(hoodieRecords.iterator()).produce(queue1);
|
||||
return true;
|
||||
});
|
||||
|
||||
// waiting for permits to expire.
|
||||
while (!isQueueFull(queue1.rateLimiter)) {
|
||||
Thread.sleep(10);
|
||||
}
|
||||
// notify queueing thread of an exception and ensure that it exits.
|
||||
final Exception e = new Exception("Failing it :)");
|
||||
queue1.markAsFailed(e);
|
||||
final Throwable thrown1 = assertThrows(ExecutionException.class, resFuture::get,
|
||||
"exception is expected");
|
||||
assertEquals(HoodieException.class, thrown1.getCause().getClass());
|
||||
assertEquals(e, thrown1.getCause().getCause());
|
||||
|
||||
// second let us raise an exception while doing record queueing. this exception should get
|
||||
// propagated to
|
||||
// queue iterator reader.
|
||||
final RuntimeException expectedException = new RuntimeException("failing record reading");
|
||||
final Iterator<HoodieRecord> mockHoodieRecordsIterator = mock(Iterator.class);
|
||||
when(mockHoodieRecordsIterator.hasNext()).thenReturn(true);
|
||||
when(mockHoodieRecordsIterator.next()).thenThrow(expectedException);
|
||||
BoundedInMemoryQueue<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>> queue2 =
|
||||
new BoundedInMemoryQueue(memoryLimitInBytes, getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
|
||||
// Produce
|
||||
Future<Boolean> res = executorService.submit(() -> {
|
||||
try {
|
||||
new IteratorBasedQueueProducer<>(mockHoodieRecordsIterator).produce(queue2);
|
||||
} catch (Exception ex) {
|
||||
queue2.markAsFailed(ex);
|
||||
throw ex;
|
||||
}
|
||||
return true;
|
||||
});
|
||||
|
||||
final Throwable thrown2 = assertThrows(Exception.class, () -> {
|
||||
queue2.iterator().hasNext();
|
||||
}, "exception is expected");
|
||||
assertEquals(expectedException, thrown2.getCause());
|
||||
// queueing thread should also have exited. make sure that it is not running.
|
||||
final Throwable thrown3 = assertThrows(ExecutionException.class, res::get,
|
||||
"exception is expected");
|
||||
assertEquals(expectedException, thrown3.getCause());
|
||||
}
|
||||
|
||||
private boolean isQueueFull(Semaphore rateLimiter) {
|
||||
return (rateLimiter.availablePermits() == 0 && rateLimiter.hasQueuedThreads());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,100 @@
|
||||
/*
|
||||
* 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.execution;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryQueueConsumer;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.execution.HoodieLazyInsertIterable.getTransformFunction;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestSparkBoundedInMemoryExecutor extends HoodieClientTestHarness {
|
||||
|
||||
private final String instantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExecutor() {
|
||||
|
||||
final List<HoodieRecord> hoodieRecords = dataGen.generateInserts(instantTime, 100);
|
||||
|
||||
HoodieWriteConfig hoodieWriteConfig = mock(HoodieWriteConfig.class);
|
||||
when(hoodieWriteConfig.getWriteBufferLimitBytes()).thenReturn(1024);
|
||||
BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer> consumer =
|
||||
new BoundedInMemoryQueueConsumer<HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord>, Integer>() {
|
||||
|
||||
private int count = 0;
|
||||
|
||||
@Override
|
||||
protected void consumeOneRecord(HoodieLazyInsertIterable.HoodieInsertValueGenResult<HoodieRecord> record) {
|
||||
count++;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void finish() {
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Integer getResult() {
|
||||
return count;
|
||||
}
|
||||
};
|
||||
|
||||
SparkBoundedInMemoryExecutor<HoodieRecord, Tuple2<HoodieRecord, Option<IndexedRecord>>, Integer> executor = null;
|
||||
try {
|
||||
executor = new SparkBoundedInMemoryExecutor(hoodieWriteConfig, hoodieRecords.iterator(), consumer,
|
||||
getTransformFunction(HoodieTestDataGenerator.AVRO_SCHEMA));
|
||||
int result = executor.execute();
|
||||
// It should buffer and write 100 records
|
||||
assertEquals(100, result);
|
||||
// There should be no remaining records in the buffer
|
||||
assertFalse(executor.isRemaining());
|
||||
} finally {
|
||||
if (executor != null) {
|
||||
executor.shutdownNow();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* 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.execution.bulkinsert;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestBulkInsertInternalPartitioner extends HoodieClientTestBase {
|
||||
|
||||
public static JavaRDD<HoodieRecord> generateTestRecordsForBulkInsert(JavaSparkContext jsc) {
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator();
|
||||
// RDD partition 1
|
||||
List<HoodieRecord> records1 = dataGenerator.generateInserts("0", 100);
|
||||
// RDD partition 2
|
||||
List<HoodieRecord> records2 = dataGenerator.generateInserts("0", 150);
|
||||
return jsc.parallelize(records1, 1).union(jsc.parallelize(records2, 1));
|
||||
}
|
||||
|
||||
public static Map<String, Long> generateExpectedPartitionNumRecords(JavaRDD<HoodieRecord> records) {
|
||||
return records.map(record -> record.getPartitionPath()).countByValue();
|
||||
}
|
||||
|
||||
private static JavaRDD<HoodieRecord> generateTripleTestRecordsForBulkInsert(JavaSparkContext jsc)
|
||||
throws Exception {
|
||||
return generateTestRecordsForBulkInsert(jsc).union(generateTestRecordsForBulkInsert(jsc))
|
||||
.union(generateTestRecordsForBulkInsert(jsc));
|
||||
}
|
||||
|
||||
private static Stream<Arguments> configParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{BulkInsertSortMode.GLOBAL_SORT, true, true},
|
||||
{BulkInsertSortMode.PARTITION_SORT, false, true},
|
||||
{BulkInsertSortMode.NONE, false, false}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
private void verifyRecordAscendingOrder(List<HoodieRecord> records) {
|
||||
List<HoodieRecord> expectedRecords = new ArrayList<>(records);
|
||||
Collections.sort(expectedRecords, Comparator.comparing(o -> (o.getPartitionPath() + "+" + o.getRecordKey())));
|
||||
assertEquals(expectedRecords, records);
|
||||
}
|
||||
|
||||
private void testBulkInsertInternalPartitioner(BulkInsertPartitioner partitioner,
|
||||
JavaRDD<HoodieRecord> records,
|
||||
boolean isGloballySorted, boolean isLocallySorted,
|
||||
Map<String, Long> expectedPartitionNumRecords) {
|
||||
int numPartitions = 2;
|
||||
JavaRDD<HoodieRecord> actualRecords = (JavaRDD<HoodieRecord>) partitioner.repartitionRecords(records, numPartitions);
|
||||
assertEquals(numPartitions, actualRecords.getNumPartitions());
|
||||
List<HoodieRecord> collectedActualRecords = actualRecords.collect();
|
||||
if (isGloballySorted) {
|
||||
// Verify global order
|
||||
verifyRecordAscendingOrder(collectedActualRecords);
|
||||
} else if (isLocallySorted) {
|
||||
// Verify local order
|
||||
actualRecords.mapPartitions(partition -> {
|
||||
List<HoodieRecord> partitionRecords = new ArrayList<>();
|
||||
partition.forEachRemaining(partitionRecords::add);
|
||||
verifyRecordAscendingOrder(partitionRecords);
|
||||
return Collections.emptyList().iterator();
|
||||
}).collect();
|
||||
}
|
||||
|
||||
// Verify number of records per partition path
|
||||
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||
for (HoodieRecord record : collectedActualRecords) {
|
||||
String partitionPath = record.getPartitionPath();
|
||||
actualPartitionNumRecords.put(partitionPath,
|
||||
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + 1);
|
||||
}
|
||||
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "[{index}] {0}")
|
||||
@MethodSource("configParams")
|
||||
public void testBulkInsertInternalPartitioner(BulkInsertSortMode sortMode,
|
||||
boolean isGloballySorted, boolean isLocallySorted)
|
||||
throws Exception {
|
||||
JavaRDD<HoodieRecord> records1 = generateTestRecordsForBulkInsert(jsc);
|
||||
JavaRDD<HoodieRecord> records2 = generateTripleTestRecordsForBulkInsert(jsc);
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records1, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records1));
|
||||
testBulkInsertInternalPartitioner(BulkInsertInternalPartitionerFactory.get(sortMode),
|
||||
records2, isGloballySorted, isLocallySorted, generateExpectedPartitionNumRecords(records2));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,433 @@
|
||||
/*
|
||||
* 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.index;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.Assertions;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieIndex.class, "/exampleSchema.txt", true);
|
||||
private final Random random = new Random();
|
||||
private IndexType indexType;
|
||||
private HoodieIndex index;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
private void setUp(IndexType indexType) throws Exception {
|
||||
this.indexType = indexType;
|
||||
initResources();
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.build()).withAutoCommit(false).build();
|
||||
writeClient = getHoodieWriteClient(config);
|
||||
this.index = writeClient.getIndex();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testSimpleTagLocationAndUpdate(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = "001";
|
||||
int totalRecords = 10 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
|
||||
// Insert totalRecords records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, index should not tag them since it was a failed
|
||||
// commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size() == 0);
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, index should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> record.isCurrentLocationKnown()).collect().size());
|
||||
assertEquals(totalRecords, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testTagLocationAndDuplicateUpdate(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = "001";
|
||||
int totalRecords = 10 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
JavaRDD<HoodieRecord> javaRDD1 = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
assertEquals(totalRecords, javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size());
|
||||
assertEquals(totalRecords, javaRDD.map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(totalRecords, javaRDD.filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
javaRDD.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry.getRecordKey()), entry.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE"})
|
||||
public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
int totalRecords = 20 + random.nextInt(20);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
JavaRDD<HoodieRecord> javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == totalRecords);
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation().getFileId() == null).collect().size() == 0);
|
||||
List<String> taggedFileIds = javaRDD.map(record -> record.getCurrentLocation().getFileId()).distinct().collect();
|
||||
|
||||
Map<String, String> recordKeyToPartitionPathMap = new HashMap();
|
||||
List<HoodieRecord> hoodieRecords = writeRecords.collect();
|
||||
hoodieRecords.forEach(entry -> recordKeyToPartitionPathMap.put(entry.getRecordKey(), entry.getPartitionPath()));
|
||||
|
||||
JavaRDD<HoodieKey> hoodieKeyJavaRDD = writeRecords.map(entry -> entry.getKey());
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(hoodieKeyJavaRDD, hoodieTable);
|
||||
List<HoodieKey> hoodieKeys = hoodieKeyJavaRDD.collect();
|
||||
assertEquals(totalRecords, recordLocations.collect().size());
|
||||
assertEquals(totalRecords, recordLocations.map(record -> record._1).distinct().count());
|
||||
recordLocations.foreach(entry -> assertTrue(hoodieKeys.contains(entry._1), "Missing HoodieKey"));
|
||||
recordLocations.foreach(entry -> assertEquals(recordKeyToPartitionPathMap.get(entry._1.getRecordKey()), entry._1.getPartitionPath(), "PartitionPath mismatch"));
|
||||
|
||||
// both lists should match
|
||||
assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds));
|
||||
// Rollback the last commit
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
javaRDD = (JavaRDD<HoodieRecord>) index.tagLocation(writeRecords, context, hoodieTable);
|
||||
assert (javaRDD.filter(HoodieRecord::isCurrentLocationKnown).collect().size() == 0);
|
||||
assert (javaRDD.filter(record -> record.getCurrentLocation() != null).collect().size() == 0);
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "SIMPLE",})
|
||||
public void testTagLocationAndFetchRecordLocations(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
String p1 = "2016/01/31";
|
||||
String p2 = "2015/01/31";
|
||||
String rowKey1 = UUID.randomUUID().toString();
|
||||
String rowKey2 = UUID.randomUUID().toString();
|
||||
String rowKey3 = UUID.randomUUID().toString();
|
||||
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// place same row key under a different partition.
|
||||
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
String fileId1 = testTable.addCommit("001").withInserts(p1, record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts(p1, record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts(p2, record4);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getRecordKey().equals(rowKey1)) {
|
||||
if (record.getPartitionPath().equals(p2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
} else {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
}
|
||||
} else if (record.getRecordKey().equals(rowKey2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId2);
|
||||
} else if (record.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocations = getRecordLocations(recordRDD.map(HoodieRecord::getKey), hoodieTable);
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> entry : recordLocations.collect()) {
|
||||
if (entry._1.getRecordKey().equals(rowKey1)) {
|
||||
assertTrue(entry._2.isPresent(), "Row1 should have been present ");
|
||||
if (entry._1.getPartitionPath().equals(p2)) {
|
||||
assertTrue(entry._2.isPresent(), "Row1 should have been present ");
|
||||
assertEquals(entry._2.get().getRight(), fileId3);
|
||||
} else {
|
||||
assertEquals(entry._2.get().getRight(), fileId1);
|
||||
}
|
||||
} else if (entry._1.getRecordKey().equals(rowKey2)) {
|
||||
assertTrue(entry._2.isPresent(), "Row2 should have been present ");
|
||||
assertEquals(entry._2.get().getRight(), fileId2);
|
||||
} else if (entry._1.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(entry._2.isPresent(), "Row3 should have been absent ");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"GLOBAL_SIMPLE"})
|
||||
public void testSimpleGlobalIndexTagLocationWhenShouldUpdatePartitionPath(IndexType indexType) throws Exception {
|
||||
setUp(indexType);
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
|
||||
.withGlobalSimpleIndexUpdatePartitionPath(true)
|
||||
.withBloomIndexUpdatePartitionPath(true)
|
||||
.build()).build();
|
||||
writeClient = getHoodieWriteClient(config);
|
||||
index = writeClient.getIndex();
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
final String p1 = "2016/01/31";
|
||||
final String p2 = "2016/02/28";
|
||||
|
||||
// Create the original partition, and put a record, along with the meta file
|
||||
// "2016/01/31": 1 file (1_0_20160131101010.parquet)
|
||||
// this record will be saved in table and will be tagged to an empty record
|
||||
RawTripTestPayload originalPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord originalRecord =
|
||||
new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()),
|
||||
originalPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord but different time so different partition
|
||||
Because GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should
|
||||
- tag the original partition of the originalRecord to an empty record for deletion, and
|
||||
- tag the new partition of the incomingRecord
|
||||
*/
|
||||
RawTripTestPayload incomingPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord incomingRecord =
|
||||
new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()),
|
||||
incomingPayload);
|
||||
/*
|
||||
This record has the same record key as originalRecord and the same partition
|
||||
Though GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should just tag the original partition
|
||||
*/
|
||||
RawTripTestPayload incomingPayloadSamePartition =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}");
|
||||
HoodieRecord incomingRecordSamePartition =
|
||||
new HoodieRecord(
|
||||
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
// We have some records to be tagged (two different partitions)
|
||||
testTable.addCommit("1000").withInserts(p1, originalRecord);
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getPartitionPath()) {
|
||||
case p1:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertTrue(record.getData() instanceof EmptyHoodieRecordPayload);
|
||||
break;
|
||||
case p2:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(incomingPayload.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
break;
|
||||
default:
|
||||
fail(String.format("Should not get partition path: %s", record.getPartitionPath()));
|
||||
}
|
||||
}
|
||||
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = (JavaRDD<HoodieRecord>) index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(p1, record.getPartitionPath());
|
||||
assertEquals(incomingPayloadSamePartition.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
private JavaPairRDD<HoodieKey, Option<Pair<String, String>>> getRecordLocations(JavaRDD<HoodieKey> keyRDD, HoodieTable hoodieTable) {
|
||||
JavaRDD<HoodieRecord> recordRDD = (JavaRDD<HoodieRecord>) index.tagLocation(
|
||||
keyRDD.map(k -> new HoodieRecord(k, new EmptyHoodieRecordPayload())), context, hoodieTable);
|
||||
return recordRDD.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,178 @@
|
||||
/*
|
||||
* 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.index;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIndexException;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieGlobalBloomIndex;
|
||||
import org.apache.hudi.index.hbase.SparkHoodieHBaseIndex;
|
||||
import org.apache.hudi.index.simple.SparkHoodieSimpleIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.EnumSource;
|
||||
|
||||
import java.nio.file.Path;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class TestHoodieIndexConfigs {
|
||||
|
||||
private String basePath;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp(@TempDir Path tempDir) {
|
||||
basePath = tempDir.toString();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@EnumSource(value = IndexType.class, names = {"BLOOM", "GLOBAL_BLOOM", "SIMPLE", "GLOBAL_SIMPLE", "HBASE"})
|
||||
public void testCreateIndex(IndexType indexType) throws Exception {
|
||||
HoodieWriteConfig config;
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
switch (indexType) {
|
||||
case INMEMORY:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.INMEMORY).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkInMemoryHashIndex);
|
||||
break;
|
||||
case BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieBloomIndex);
|
||||
break;
|
||||
case GLOBAL_BLOOM:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.GLOBAL_BLOOM).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieGlobalBloomIndex);
|
||||
break;
|
||||
case SIMPLE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(IndexType.SIMPLE).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieSimpleIndex);
|
||||
break;
|
||||
case HBASE:
|
||||
config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder().build()).build())
|
||||
.build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof SparkHoodieHBaseIndex);
|
||||
break;
|
||||
default:
|
||||
// no -op. just for checkstyle errors
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateDummyIndex() {
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
HoodieWriteConfig config = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(DummyHoodieIndex.class.getName()).build()).build();
|
||||
assertTrue(SparkHoodieIndex.createIndex(config) instanceof DummyHoodieIndex);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateIndexWithException() {
|
||||
HoodieWriteConfig.Builder clientConfigBuilder = HoodieWriteConfig.newBuilder();
|
||||
HoodieIndexConfig.Builder indexConfigBuilder = HoodieIndexConfig.newBuilder();
|
||||
final HoodieWriteConfig config1 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown1 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config1);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown1.getMessage().contains("is not a subclass of HoodieIndex"));
|
||||
|
||||
final HoodieWriteConfig config2 = clientConfigBuilder.withPath(basePath)
|
||||
.withIndexConfig(indexConfigBuilder.withIndexClass(IndexWithoutConstructor.class.getName()).build()).build();
|
||||
final Throwable thrown2 = assertThrows(HoodieException.class, () -> {
|
||||
SparkHoodieIndex.createIndex(config2);
|
||||
}, "exception is expected");
|
||||
assertTrue(thrown2.getMessage().contains("Unable to instantiate class"));
|
||||
}
|
||||
|
||||
public static class DummyHoodieIndex<T extends HoodieRecordPayload<T>> extends SparkHoodieIndex<T> {
|
||||
|
||||
public DummyHoodieIndex(HoodieWriteConfig config) {
|
||||
super(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> records,
|
||||
HoodieEngineContext context,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> hoodieTable) throws HoodieIndexException {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rollbackCommit(String instantTime) {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isGlobal() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canIndexLogFiles() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isImplicitWithStorage() {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static class IndexWithConstructor {
|
||||
|
||||
public IndexWithConstructor(HoodieWriteConfig config) {
|
||||
}
|
||||
}
|
||||
|
||||
public static class IndexWithoutConstructor {
|
||||
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestBucketizedBloomCheckPartitioner {
|
||||
|
||||
@Test
|
||||
public void testAssignmentCorrectness() {
|
||||
Map<String, Long> fileToComparisons = new HashMap<String, Long>() {
|
||||
{
|
||||
put("f1", 40L);
|
||||
put("f2", 35L);
|
||||
put("f3", 20L);
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(4, fileToComparisons, 10);
|
||||
Map<String, List<Integer>> assignments = p.getFileGroupToPartitions();
|
||||
assertEquals(4, assignments.get("f1").size(), "f1 should have 4 buckets");
|
||||
assertEquals(4, assignments.get("f2").size(), "f2 should have 4 buckets");
|
||||
assertEquals(2, assignments.get("f3").size(), "f3 should have 2 buckets");
|
||||
assertArrayEquals(new Integer[] {0, 0, 1, 3}, assignments.get("f1").toArray(), "f1 spread across 3 partitions");
|
||||
assertArrayEquals(new Integer[] {1, 2, 2, 0}, assignments.get("f2").toArray(), "f2 spread across 3 partitions");
|
||||
assertArrayEquals(new Integer[] {3, 1}, assignments.get("f3").toArray(), "f3 spread across 2 partitions");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUniformPacking() {
|
||||
// evenly distribute 10 buckets/file across 100 partitions
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner partitioner = new BucketizedBloomCheckPartitioner(100, comparisons1, 10);
|
||||
Map<String, List<Integer>> assignments = partitioner.getFileGroupToPartitions();
|
||||
assignments.forEach((key, value) -> assertEquals(10, value.size()));
|
||||
Map<Integer, Long> partitionToNumBuckets =
|
||||
assignments.entrySet().stream().flatMap(e -> e.getValue().stream().map(p -> Pair.of(p, e.getKey())))
|
||||
.collect(Collectors.groupingBy(Pair::getLeft, Collectors.counting()));
|
||||
partitionToNumBuckets.forEach((key, value) -> assertEquals(1L, value.longValue()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNumPartitions() {
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 10).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(10000, comparisons1, 10);
|
||||
assertEquals(100, p.numPartitions(), "num partitions must equal total buckets");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetPartitions() {
|
||||
Map<String, Long> comparisons1 = new HashMap<String, Long>() {
|
||||
{
|
||||
IntStream.range(0, 100000).forEach(f -> put("f" + f, 100L));
|
||||
}
|
||||
};
|
||||
BucketizedBloomCheckPartitioner p = new BucketizedBloomCheckPartitioner(1000, comparisons1, 10);
|
||||
|
||||
IntStream.range(0, 100000).forEach(f -> {
|
||||
int partition = p.getPartition(Pair.of("f" + f, "value"));
|
||||
assertTrue(0 <= partition && partition <= 1000, "partition is out of range: " + partition);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,458 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
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.io.HoodieKeyLookupHandle;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.nio.file.Paths;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieBloomIndex.class, "/exampleSchema.txt", true);
|
||||
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with rangePruning={0}, treeFiltering={1}, bucketizedChecking={2}";
|
||||
|
||||
public static Stream<Arguments> configParams() {
|
||||
Object[][] data =
|
||||
new Object[][] {{true, true, true}, {false, true, true}, {true, true, false}, {true, false, true}};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initFileSystem();
|
||||
// We have some records to be tagged (two different partitions)
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeConfig(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().bloomIndexPruneByRanges(rangePruning)
|
||||
.bloomIndexTreebasedFilter(treeFiltering).bloomIndexBucketizedChecking(bucketizedChecking)
|
||||
.bloomIndexKeysPerBucket(2).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testLoadInvolvedFiles(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
|
||||
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
|
||||
.withInserts("2015/03/12", "3", record1)
|
||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||
|
||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
assertEquals(4, filesList.size());
|
||||
|
||||
if (rangePruning) {
|
||||
// these files will not have the key ranges
|
||||
assertNull(filesList.get(0)._2().getMaxRecordKey());
|
||||
assertNull(filesList.get(0)._2().getMinRecordKey());
|
||||
assertFalse(filesList.get(1)._2().hasKeyRanges());
|
||||
assertNotNull(filesList.get(2)._2().getMaxRecordKey());
|
||||
assertNotNull(filesList.get(2)._2().getMinRecordKey());
|
||||
assertTrue(filesList.get(3)._2().hasKeyRanges());
|
||||
|
||||
// no longer sorted, but should have same files.
|
||||
|
||||
List<Tuple2<String, BloomIndexFileInfo>> expected =
|
||||
Arrays.asList(new Tuple2<>("2016/04/01", new BloomIndexFileInfo("2")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("1")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("3", "000", "000")),
|
||||
new Tuple2<>("2015/03/12", new BloomIndexFileInfo("4", "001", "003")));
|
||||
assertEquals(expected, filesList);
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testRangePruning(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
SparkHoodieBloomIndex index = new SparkHoodieBloomIndex(config);
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22",
|
||||
Arrays.asList(new BloomIndexFileInfo("f1"), new BloomIndexFileInfo("f2", "000", "000"),
|
||||
new BloomIndexFileInfo("f3", "001", "003"), new BloomIndexFileInfo("f4", "002", "007"),
|
||||
new BloomIndexFileInfo("f5", "009", "010")));
|
||||
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/22", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/22", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(t -> t._1, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f3", "f4")), new HashSet<>(recordKeyToFileComps.get("003")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("004")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f1", "f4")), new HashSet<>(recordKeyToFileComps.get("005")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckUUIDsAgainstOneFile() throws Exception {
|
||||
final String partition = "2016/01/31";
|
||||
// Create some records to use
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
String recordStr4 = "{\"_row_key\":\"4eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// We write record1, record2 to a parquet file, but the bloom filter contains (record1,
|
||||
// record2, record3).
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record3.getRecordKey());
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
|
||||
String fileId = testTable.addCommit("000").withInserts(partition, record1, record2);
|
||||
String filename = testTable.getBaseFileNameById(fileId);
|
||||
|
||||
// The bloom filter contains 3 records
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record3.getRecordKey()));
|
||||
assertFalse(filter.mightContain(record4.getRecordKey()));
|
||||
|
||||
// Compare with file
|
||||
List<String> uuids =
|
||||
Arrays.asList(record1.getRecordKey(), record2.getRecordKey(), record3.getRecordKey(), record4.getRecordKey());
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieKeyLookupHandle keyHandle = new HoodieKeyLookupHandle<>(config, table, Pair.of(partition, fileId));
|
||||
List<String> results = keyHandle.checkCandidatesAgainstFile(hadoopConf, uuids,
|
||||
new Path(Paths.get(basePath, partition, filename).toString()));
|
||||
assertEquals(results.size(), 2);
|
||||
assertTrue(results.get(0).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")
|
||||
|| results.get(1).equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0"));
|
||||
assertTrue(results.get(0).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")
|
||||
|| results.get(1).equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0"));
|
||||
// TODO(vc): Need more coverage on actual filenames
|
||||
// assertTrue(results.get(0)._2().equals(filename));
|
||||
// assertTrue(results.get(1)._2().equals(filename));
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testTagLocationWithEmptyRDD(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.emptyRDD();
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
|
||||
assertDoesNotThrow(() -> {
|
||||
bloomIndex.tagLocation(recordRDD, context, table);
|
||||
}, "EmptyRDD should not result in IllegalArgumentException: Positive number of slices required");
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testTagLocation(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
String rowKey1 = UUID.randomUUID().toString();
|
||||
String rowKey2 = UUID.randomUUID().toString();
|
||||
String rowKey3 = UUID.randomUUID().toString();
|
||||
String recordStr1 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"" + rowKey2 + "\",\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"" + rowKey3 + "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// place same row key under a different partition.
|
||||
String recordStr4 = "{\"_row_key\":\"" + rowKey1 + "\",\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
HoodieSparkTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
// Should not find any files
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String fileId1 = testTable.addCommit("001").withInserts("2016/01/31", record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts("2016/01/31", record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4);
|
||||
|
||||
// We do the tag again
|
||||
taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, HoodieSparkTable.create(config, context, metaClient));
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getRecordKey().equals(rowKey1)) {
|
||||
if (record.getPartitionPath().equals("2015/01/31")) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
} else {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
}
|
||||
} else if (record.getRecordKey().equals(rowKey2)) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId2);
|
||||
} else if (record.getRecordKey().equals(rowKey3)) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testCheckExists(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have some records to be tagged (two different partitions)
|
||||
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"3eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
// record key same as recordStr2
|
||||
String recordStr4 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2015-01-31T03:16:41.415Z\",\"number\":32}";
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieKey key1 = new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath());
|
||||
HoodieRecord record1 = new HoodieRecord(key1, rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieKey key2 = new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath());
|
||||
HoodieRecord record2 = new HoodieRecord(key2, rowChange2);
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
HoodieKey key3 = new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath());
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieKey key4 = new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath());
|
||||
HoodieRecord record4 = new HoodieRecord(key4, rowChange4);
|
||||
JavaRDD<HoodieKey> keysRDD = jsc.parallelize(Arrays.asList(key1, key2, key3, key4));
|
||||
|
||||
// Also create the metadata and config
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Let's tag
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
|
||||
// Should not find any files
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
|
||||
assertTrue(!record._2.isPresent());
|
||||
}
|
||||
|
||||
// We create three parquet file, each having one record. (two different partitions)
|
||||
String fileId1 = testTable.addCommit("001").withInserts("2016/01/31", record1);
|
||||
String fileId2 = testTable.addCommit("002").withInserts("2016/01/31", record2);
|
||||
String fileId3 = testTable.addCommit("003").withInserts("2015/01/31", record4);
|
||||
|
||||
// We do the tag again
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
taggedRecords = bloomIndex.tagLocation(keysRDD.map(k -> new HoodieRecord(k, null)), context, hoodieTable);
|
||||
recordLocationsRDD = taggedRecords
|
||||
.mapToPair(hr -> new Tuple2<>(hr.getKey(), hr.isCurrentLocationKnown()
|
||||
? Option.of(Pair.of(hr.getPartitionPath(), hr.getCurrentLocation().getFileId()))
|
||||
: Option.empty())
|
||||
);
|
||||
|
||||
// Check results
|
||||
for (Tuple2<HoodieKey, Option<Pair<String, String>>> record : recordLocationsRDD.collect()) {
|
||||
if (record._1.getRecordKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
assertEquals(fileId1, record._2.get().getRight());
|
||||
} else if (record._1.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertTrue(record._2.isPresent());
|
||||
if (record._1.getPartitionPath().equals("2015/01/31")) {
|
||||
assertEquals(fileId3, record._2.get().getRight());
|
||||
} else {
|
||||
assertEquals(fileId2, record._2.get().getRight());
|
||||
}
|
||||
} else if (record._1.getRecordKey().equals("3eb5b87c-1fej-4edd-87b4-6ec96dc405a0")) {
|
||||
assertFalse(record._2.isPresent());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testBloomFilterFalseError(boolean rangePruning, boolean treeFiltering, boolean bucketizedChecking) throws Exception {
|
||||
// We have two hoodie records
|
||||
String recordStr1 = "{\"_row_key\":\"1eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"2eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
|
||||
// We write record1 to a parquet file, using a bloom filter having both records
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(10000, 0.0000001, -1,
|
||||
BloomFilterTypeCode.SIMPLE.name());
|
||||
filter.add(record2.getRecordKey());
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(metaClient, SCHEMA, filter);
|
||||
String fileId = testTable.addCommit("000").withInserts("2016/01/31", record1);
|
||||
assertTrue(filter.mightContain(record1.getRecordKey()));
|
||||
assertTrue(filter.mightContain(record2.getRecordKey()));
|
||||
|
||||
// We do the tag
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2));
|
||||
HoodieWriteConfig config = makeConfig(rangePruning, treeFiltering, bucketizedChecking);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
SparkHoodieBloomIndex bloomIndex = new SparkHoodieBloomIndex(config);
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = bloomIndex.tagLocation(recordRDD, context, table);
|
||||
|
||||
// Check results
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
if (record.getKey().equals("1eb5b87a-1feh-4edd-87b4-6ec96dc405a0")) {
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId);
|
||||
} else if (record.getRecordKey().equals("2eb5b87b-1feu-4edd-87b4-6ec96dc405a0")) {
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,345 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestHoodieGlobalBloomIndex extends HoodieClientTestHarness {
|
||||
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestHoodieGlobalBloomIndex.class, "/exampleSchema.txt", true);
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLoadInvolvedFiles() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files, along with the meta file
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
List<String> partitions = Arrays.asList("2016/01/21", "2016/04/01");
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
List<Tuple2<String, BloomIndexFileInfo>> filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
// Still 0, as no valid commit
|
||||
assertEquals(0, filesList.size());
|
||||
|
||||
testTable.addCommit("20160401010101").withInserts("2016/04/01", "2");
|
||||
testTable.addCommit("20150312101010").withInserts("2015/03/12", "1")
|
||||
.withInserts("2015/03/12", "3", record1)
|
||||
.withInserts("2015/03/12", "4", record2, record3, record4);
|
||||
|
||||
filesList = index.loadInvolvedFiles(partitions, context, hoodieTable);
|
||||
assertEquals(4, filesList.size());
|
||||
|
||||
Map<String, BloomIndexFileInfo> filesMap = toFileMap(filesList);
|
||||
// key ranges checks
|
||||
assertNull(filesMap.get("2016/04/01/2").getMaxRecordKey());
|
||||
assertNull(filesMap.get("2016/04/01/2").getMinRecordKey());
|
||||
assertFalse(filesMap.get("2015/03/12/1").hasKeyRanges());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMaxRecordKey());
|
||||
assertNotNull(filesMap.get("2015/03/12/3").getMinRecordKey());
|
||||
assertTrue(filesMap.get("2015/03/12/3").hasKeyRanges());
|
||||
|
||||
Map<String, BloomIndexFileInfo> expected = new HashMap<>();
|
||||
expected.put("2016/04/01/2", new BloomIndexFileInfo("2"));
|
||||
expected.put("2015/03/12/1", new BloomIndexFileInfo("1"));
|
||||
expected.put("2015/03/12/3", new BloomIndexFileInfo("3", "000", "000"));
|
||||
expected.put("2015/03/12/4", new BloomIndexFileInfo("4", "001", "003"));
|
||||
|
||||
assertEquals(expected, filesMap);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testExplodeRecordRDDWithFileComparisons() {
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
|
||||
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo = new HashMap<>();
|
||||
partitionToFileIndexInfo.put("2017/10/22", Arrays.asList(new BloomIndexFileInfo("f1"),
|
||||
new BloomIndexFileInfo("f2", "000", "000"), new BloomIndexFileInfo("f3", "001", "003")));
|
||||
|
||||
partitionToFileIndexInfo.put("2017/10/23",
|
||||
Arrays.asList(new BloomIndexFileInfo("f4", "002", "007"), new BloomIndexFileInfo("f5", "009", "010")));
|
||||
|
||||
// the partition of the key of the incoming records will be ignored
|
||||
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
|
||||
jsc.parallelize(Arrays.asList(new Tuple2<>("2017/10/21", "003"), new Tuple2<>("2017/10/22", "002"),
|
||||
new Tuple2<>("2017/10/22", "005"), new Tuple2<>("2017/10/23", "004"))).mapToPair(t -> t);
|
||||
|
||||
List<Tuple2<String, HoodieKey>> comparisonKeyList =
|
||||
index.explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD).collect();
|
||||
|
||||
/*
|
||||
* expecting: f4, HoodieKey { recordKey=003 partitionPath=2017/10/23} f1, HoodieKey { recordKey=003
|
||||
* partitionPath=2017/10/22} f3, HoodieKey { recordKey=003 partitionPath=2017/10/22} f4, HoodieKey { recordKey=002
|
||||
* partitionPath=2017/10/23} f1, HoodieKey { recordKey=002 partitionPath=2017/10/22} f3, HoodieKey { recordKey=002
|
||||
* partitionPath=2017/10/22} f4, HoodieKey { recordKey=005 partitionPath=2017/10/23} f1, HoodieKey { recordKey=005
|
||||
* partitionPath=2017/10/22} f4, HoodieKey { recordKey=004 partitionPath=2017/10/23} f1, HoodieKey { recordKey=004
|
||||
* partitionPath=2017/10/22}
|
||||
*/
|
||||
assertEquals(10, comparisonKeyList.size());
|
||||
|
||||
Map<String, List<String>> recordKeyToFileComps = comparisonKeyList.stream()
|
||||
.collect(Collectors.groupingBy(t -> t._2.getRecordKey(), Collectors.mapping(Tuple2::_1, Collectors.toList())));
|
||||
|
||||
assertEquals(4, recordKeyToFileComps.size());
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("002")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1", "f3")), new HashSet<>(recordKeyToFileComps.get("003")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("004")));
|
||||
assertEquals(new HashSet<>(Arrays.asList("f4", "f1")), new HashSet<>(recordKeyToFileComps.get("005")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocation() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
|
||||
// Create some partitions, and put some files, along with the meta file
|
||||
// "2016/01/21": 0 file
|
||||
// "2016/04/01": 1 file (2_0_20160401010101.parquet)
|
||||
// "2015/03/12": 3 files (1_0_20150312101010.parquet, 3_0_20150312101010.parquet, 4_0_20150312101010.parquet)
|
||||
testTable.withPartitionMetaFiles("2016/01/21", "2016/04/01", "2015/03/12");
|
||||
|
||||
RawTripTestPayload rowChange1 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1);
|
||||
RawTripTestPayload rowChange2 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"001\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record2 =
|
||||
new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2);
|
||||
RawTripTestPayload rowChange3 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"002\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record3 =
|
||||
new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3);
|
||||
|
||||
// this record will be saved in table and will be tagged to the incoming record5
|
||||
RawTripTestPayload rowChange4 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record4 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
// this has the same record key as record4 but different time so different partition, but globalbloomIndex should
|
||||
// tag the original partition of the saved record4
|
||||
RawTripTestPayload rowChange5 =
|
||||
new RawTripTestPayload("{\"_row_key\":\"003\",\"time\":\"2016-02-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord record5 =
|
||||
new HoodieRecord(new HoodieKey(rowChange5.getRowKey(), rowChange5.getPartitionPath()), rowChange5);
|
||||
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Arrays.asList(record1, record2, record3, record5));
|
||||
|
||||
// intentionally missed the partition "2015/03/12" to see if the GlobalBloomIndex can pick it up
|
||||
String fileId1 = testTable.addCommit("1000").withInserts("2016/04/01", record1);
|
||||
String fileId2 = testTable.addCommit("2000").withInserts("2015/03/12");
|
||||
String fileId3 = testTable.addCommit("3000").withInserts("2015/03/12", record2);
|
||||
String fileId4 = testTable.addCommit("4000").withInserts("2015/03/12", record4);
|
||||
|
||||
// partitions will NOT be respected by this loadInvolvedFiles(...) call
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getRecordKey()) {
|
||||
case "000":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId1);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange1.getJsonData());
|
||||
break;
|
||||
case "001":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId3);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange2.getJsonData());
|
||||
break;
|
||||
case "002":
|
||||
assertFalse(record.isCurrentLocationKnown());
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange3.getJsonData());
|
||||
break;
|
||||
case "003":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId4);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange5.getJsonData());
|
||||
break;
|
||||
case "004":
|
||||
assertEquals(record.getCurrentLocation().getFileId(), fileId4);
|
||||
assertEquals(((RawTripTestPayload) record.getData()).getJsonData(), rowChange4.getJsonData());
|
||||
break;
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown Key: " + record.getRecordKey());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationWhenShouldUpdatePartitionPath() throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath)
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withBloomIndexUpdatePartitionPath(true).build())
|
||||
.build();
|
||||
SparkHoodieGlobalBloomIndex index = new SparkHoodieGlobalBloomIndex(config);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, SCHEMA);
|
||||
final String p1 = "2016/01/31";
|
||||
final String p2 = "2016/02/28";
|
||||
|
||||
// Create the original partition, and put a record, along with the meta file
|
||||
// "2016/01/31": 1 file (1_0_20160131101010.parquet)
|
||||
// this record will be saved in table and will be tagged to an empty record
|
||||
RawTripTestPayload originalPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord originalRecord =
|
||||
new HoodieRecord(new HoodieKey(originalPayload.getRowKey(), originalPayload.getPartitionPath()),
|
||||
originalPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord but different time so different partition
|
||||
Because GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should
|
||||
- tag the original partition of the originalRecord to an empty record for deletion, and
|
||||
- tag the new partition of the incomingRecord
|
||||
*/
|
||||
RawTripTestPayload incomingPayload =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-02-28T03:16:41.415Z\",\"number\":12}");
|
||||
HoodieRecord incomingRecord =
|
||||
new HoodieRecord(new HoodieKey(incomingPayload.getRowKey(), incomingPayload.getPartitionPath()),
|
||||
incomingPayload);
|
||||
|
||||
/*
|
||||
This record has the same record key as originalRecord and the same partition
|
||||
Though GLOBAL_BLOOM_INDEX_SHOULD_UPDATE_PARTITION_PATH = true,
|
||||
globalBloomIndex should just tag the original partition
|
||||
*/
|
||||
RawTripTestPayload incomingPayloadSamePartition =
|
||||
new RawTripTestPayload("{\"_row_key\":\"000\",\"time\":\"2016-01-31T04:16:41.415Z\",\"number\":15}");
|
||||
HoodieRecord incomingRecordSamePartition =
|
||||
new HoodieRecord(
|
||||
new HoodieKey(incomingPayloadSamePartition.getRowKey(), incomingPayloadSamePartition.getPartitionPath()),
|
||||
incomingPayloadSamePartition);
|
||||
|
||||
testTable.addCommit("1000").withInserts(p1, originalRecord);
|
||||
|
||||
// test against incoming record with a different partition
|
||||
JavaRDD<HoodieRecord> recordRDD = jsc.parallelize(Collections.singletonList(incomingRecord));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDD = index.tagLocation(recordRDD, context, hoodieTable);
|
||||
|
||||
assertEquals(2, taggedRecordRDD.count());
|
||||
for (HoodieRecord record : taggedRecordRDD.collect()) {
|
||||
switch (record.getPartitionPath()) {
|
||||
case p1:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertTrue(record.getData() instanceof EmptyHoodieRecordPayload);
|
||||
break;
|
||||
case p2:
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(incomingPayload.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
break;
|
||||
default:
|
||||
fail(String.format("Should not get partition path: %s", record.getPartitionPath()));
|
||||
}
|
||||
}
|
||||
|
||||
// test against incoming record with the same partition
|
||||
JavaRDD<HoodieRecord> recordRDDSamePartition = jsc
|
||||
.parallelize(Collections.singletonList(incomingRecordSamePartition));
|
||||
JavaRDD<HoodieRecord> taggedRecordRDDSamePartition = index.tagLocation(recordRDDSamePartition, context, hoodieTable);
|
||||
|
||||
assertEquals(1, taggedRecordRDDSamePartition.count());
|
||||
HoodieRecord record = taggedRecordRDDSamePartition.first();
|
||||
assertEquals("000", record.getRecordKey());
|
||||
assertEquals(p1, record.getPartitionPath());
|
||||
assertEquals(incomingPayloadSamePartition.getJsonData(), ((RawTripTestPayload) record.getData()).getJsonData());
|
||||
}
|
||||
|
||||
// convert list to map to avoid sorting order dependencies
|
||||
private static Map<String, BloomIndexFileInfo> toFileMap(List<Tuple2<String, BloomIndexFileInfo>> filesList) {
|
||||
Map<String, BloomIndexFileInfo> filesMap = new HashMap<>();
|
||||
for (Tuple2<String, BloomIndexFileInfo> t : filesList) {
|
||||
filesMap.put(t._1() + "/" + t._2().getFileId(), t._2());
|
||||
}
|
||||
return filesMap;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,176 @@
|
||||
/*
|
||||
* 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.index.bloom;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Tests {@link KeyRangeLookupTree}.
|
||||
*/
|
||||
public class TestKeyRangeLookupTree {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
private KeyRangeLookupTree keyRangeLookupTree;
|
||||
private Map<String, HashSet<String>> expectedMatches;
|
||||
|
||||
public TestKeyRangeLookupTree() {
|
||||
keyRangeLookupTree = new KeyRangeLookupTree();
|
||||
expectedMatches = new HashMap<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for single node in the tree for different inputs.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpOneEntry() {
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(300), Long.toString(450), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
testRangeOfInputs(290, 305);
|
||||
testRangeOfInputs(390, 400);
|
||||
testRangeOfInputs(445, 455);
|
||||
testRangeOfInputs(600, 605);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for many entries in the tree with same start value and different end values.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpManyEntriesWithSameStartValue() {
|
||||
String startKey = Long.toString(120);
|
||||
long endKey = 250;
|
||||
KeyRangeNode toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
endKey += 1 + RANDOM.nextInt(100);
|
||||
toInsert = new KeyRangeNode(startKey, Long.toString(endKey), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
}
|
||||
testRangeOfInputs(110, endKey + 5);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests for many duplicte entries in the tree.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUpManyDulicateEntries() {
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
toInsert = new KeyRangeNode(Long.toString(1200), Long.toString(2000), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
}
|
||||
testRangeOfInputs(1050, 1100);
|
||||
testRangeOfInputs(1500, 1600);
|
||||
testRangeOfInputs(1990, 2100);
|
||||
}
|
||||
|
||||
// Tests helpers
|
||||
|
||||
/**
|
||||
* Tests for curated entries in look up tree.
|
||||
*/
|
||||
@Test
|
||||
public void testFileGroupLookUp() {
|
||||
|
||||
// testing with hand curated inputs
|
||||
KeyRangeNode toInsert = new KeyRangeNode(Long.toString(500), Long.toString(600), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(950), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(550), Long.toString(775), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(725), Long.toString(850), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(825), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(750), Long.toString(990), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(800), Long.toString(820), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(200), Long.toString(550), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(520), Long.toString(600), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
toInsert = new KeyRangeNode(Long.toString(120), Long.toString(620), UUID.randomUUID().toString());
|
||||
updateExpectedMatchesToTest(toInsert);
|
||||
keyRangeLookupTree.insert(toInsert);
|
||||
testRangeOfInputs(110, 999);
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to test the look up tree for different range of input keys.
|
||||
*
|
||||
* @param start starting value of the look up key
|
||||
* @param end end value of the look up tree
|
||||
*/
|
||||
private void testRangeOfInputs(long start, long end) {
|
||||
for (long i = start; i <= end; i++) {
|
||||
String iStr = Long.toString(i);
|
||||
if (!expectedMatches.containsKey(iStr)) {
|
||||
assertEquals(Collections.EMPTY_SET, keyRangeLookupTree.getMatchingIndexFiles(iStr));
|
||||
} else {
|
||||
assertEquals(expectedMatches.get(iStr), keyRangeLookupTree.getMatchingIndexFiles(iStr));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the expected matches for a given {@link KeyRangeNode}.
|
||||
*
|
||||
* @param toInsert the {@link KeyRangeNode} to be inserted
|
||||
*/
|
||||
private void updateExpectedMatchesToTest(KeyRangeNode toInsert) {
|
||||
long startKey = Long.parseLong(toInsert.getMinRecordKey());
|
||||
long endKey = Long.parseLong(toInsert.getMaxRecordKey());
|
||||
for (long i = startKey; i <= endKey; i++) {
|
||||
String iStr = Long.toString(i);
|
||||
if (!expectedMatches.containsKey(iStr)) {
|
||||
expectedMatches.put(iStr, new HashSet<>());
|
||||
}
|
||||
expectedMatches.get(iStr).add(toInsert.getFileNameList().get(0));
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,481 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.FunctionalTestHarness;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HBaseTestingUtility;
|
||||
import org.apache.hadoop.hbase.TableName;
|
||||
import org.apache.hadoop.hbase.client.Connection;
|
||||
import org.apache.hadoop.hbase.client.Get;
|
||||
import org.apache.hadoop.hbase.client.HTable;
|
||||
import org.apache.hadoop.hbase.client.Put;
|
||||
import org.apache.hadoop.hbase.client.Result;
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.MethodOrderer;
|
||||
import org.junit.jupiter.api.Tag;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.TestMethodOrder;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.atMost;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* Note :: HBaseTestingUtility is really flaky with issues where the HbaseMiniCluster fails to shutdown across tests,
|
||||
* (see one problem here : https://issues.apache.org/jira/browse/HBASE-15835). Hence, the need to use
|
||||
* {@link MethodOrderer.Alphanumeric} to make sure the tests run in order. Please alter the order of tests running carefully.
|
||||
*/
|
||||
@TestMethodOrder(MethodOrderer.Alphanumeric.class)
|
||||
@Tag("functional")
|
||||
public class TestHBaseIndex extends FunctionalTestHarness {
|
||||
|
||||
private static final String TABLE_NAME = "test_table";
|
||||
private static HBaseTestingUtility utility;
|
||||
private static Configuration hbaseConfig;
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private HoodieTestDataGenerator dataGen;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
@AfterAll
|
||||
public static void clean() throws Exception {
|
||||
if (utility != null) {
|
||||
utility.deleteTable(TABLE_NAME);
|
||||
utility.shutdownMiniCluster();
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeAll
|
||||
public static void init() throws Exception {
|
||||
// Initialize HbaseMiniCluster
|
||||
hbaseConfig = HBaseConfiguration.create();
|
||||
hbaseConfig.set("zookeeper.znode.parent", "/hudi-hbase-test");
|
||||
|
||||
utility = new HBaseTestingUtility(hbaseConfig);
|
||||
utility.startMiniCluster();
|
||||
hbaseConfig = utility.getConnection().getConfiguration();
|
||||
utility.createTable(TableName.valueOf(TABLE_NAME), Bytes.toBytes("_s"));
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
hadoopConf = jsc().hadoopConfiguration();
|
||||
hadoopConf.addResource(utility.getConfiguration());
|
||||
metaClient = getHoodieMetaClient(hadoopConf, basePath());
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleTagLocationAndUpdateCOW() throws Exception {
|
||||
testSimpleTagLocationAndUpdate(HoodieTableType.COPY_ON_WRITE);
|
||||
}
|
||||
|
||||
@Test void testSimpleTagLocationAndUpdateMOR() throws Exception {
|
||||
testSimpleTagLocationAndUpdate(HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
|
||||
public void testSimpleTagLocationAndUpdate(HoodieTableType tableType) throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath(), tableType);
|
||||
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since commit never occurred
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTagLocationAndDuplicateUpdate() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
|
||||
// Duplicate upsert and ensure correctness is maintained
|
||||
// We are trying to approximately imitate the case when the RDD is recomputed. For RDD creating, driver code is not
|
||||
// recomputed. This includes the state transitions. We need to delete the inflight instance so that subsequent
|
||||
// upsert will not run into conflicts.
|
||||
metaClient.getFs().delete(new Path(metaClient.getMetaPath(), "001.inflight"));
|
||||
|
||||
writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> taggedRecords = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(numRecords, taggedRecords.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, taggedRecords.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
final String newCommitTime = writeClient.startCommit();
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
|
||||
// check tagged records are tagged with correct fileIds
|
||||
List<String> fileIds = writeStatues.map(WriteStatus::getFileId).collect();
|
||||
assertEquals(0, records2.stream().filter(record -> record.getCurrentLocation().getFileId() == null).count());
|
||||
List<String> taggedFileIds = records2.stream().map(record -> record.getCurrentLocation().getFileId()).distinct().collect(Collectors.toList());
|
||||
|
||||
// both lists should match
|
||||
assertTrue(taggedFileIds.containsAll(fileIds) && fileIds.containsAll(taggedFileIds));
|
||||
// Rollback the last commit
|
||||
writeClient.rollback(newCommitTime);
|
||||
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a rolled
|
||||
// back commit
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(HoodieRecord::isCurrentLocationKnown).count());
|
||||
assertEquals(0, records3.stream().filter(record -> record.getCurrentLocation() != null).count());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTotalGetsBatching() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
|
||||
// Mock hbaseConnection and related entities
|
||||
Connection hbaseConnection = mock(Connection.class);
|
||||
HTable table = mock(HTable.class);
|
||||
when(hbaseConnection.getTable(TableName.valueOf(TABLE_NAME))).thenReturn(table);
|
||||
when(table.get((List<Get>) any())).thenReturn(new Result[0]);
|
||||
|
||||
// only for test, set the hbaseConnection to mocked object
|
||||
index.setHbaseConnection(hbaseConnection);
|
||||
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
// start a commit and generate test data
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Insert 250 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them
|
||||
index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
|
||||
// 3 batches should be executed given batchSize = 100 and parallelism = 1
|
||||
verify(table, times(3)).get((List<Get>) any());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testTotalPutsBatching() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
// start a commit and generate test data
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 250);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Insert 200 records
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
|
||||
// commit this upsert
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
|
||||
// Mock hbaseConnection and related entities
|
||||
Connection hbaseConnection = mock(Connection.class);
|
||||
HTable table = mock(HTable.class);
|
||||
when(hbaseConnection.getTable(TableName.valueOf(TABLE_NAME))).thenReturn(table);
|
||||
when(table.get((List<Get>) any())).thenReturn(new Result[0]);
|
||||
|
||||
// only for test, set the hbaseConnection to mocked object
|
||||
index.setHbaseConnection(hbaseConnection);
|
||||
|
||||
// Get all the files generated
|
||||
int numberOfDataFileIds = (int) writeStatues.map(status -> status.getFileId()).distinct().count();
|
||||
|
||||
index.updateLocation(writeStatues, context(), hoodieTable);
|
||||
// 3 batches should be executed given batchSize = 100 and <=numberOfDataFileIds getting updated,
|
||||
// so each fileId ideally gets updates
|
||||
verify(table, atMost(numberOfDataFileIds)).put((List<Put>) any());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsHBasePutAccessParallelism() {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
final JavaRDD<WriteStatus> writeStatusRDD = jsc().parallelize(
|
||||
Arrays.asList(getSampleWriteStatus(1, 2), getSampleWriteStatus(0, 3), getSampleWriteStatus(10, 0)), 10);
|
||||
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
|
||||
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
|
||||
assertEquals(10, writeStatusRDD.getNumPartitions());
|
||||
assertEquals(2, hbasePutAccessParallelism);
|
||||
assertEquals(11, hbaseNumPuts);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsHBasePutAccessParallelismWithNoInserts() {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
final JavaRDD<WriteStatus> writeStatusRDD =
|
||||
jsc().parallelize(Arrays.asList(getSampleWriteStatus(0, 2), getSampleWriteStatus(0, 1)), 10);
|
||||
final Tuple2<Long, Integer> tuple = index.getHBasePutAccessParallelism(writeStatusRDD);
|
||||
final int hbasePutAccessParallelism = Integer.parseInt(tuple._2.toString());
|
||||
final int hbaseNumPuts = Integer.parseInt(tuple._1.toString());
|
||||
assertEquals(10, writeStatusRDD.getNumPartitions());
|
||||
assertEquals(0, hbasePutAccessParallelism);
|
||||
assertEquals(0, hbaseNumPuts);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSmallBatchSize() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig(2);
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
// Insert 200 records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should not tag them since it was a failed
|
||||
// commit
|
||||
JavaRDD<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records2.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Now commit this & update location of records inserted and validate no errors
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDelete() throws Exception {
|
||||
final String newCommitTime = "001";
|
||||
final int numRecords = 10;
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, numRecords);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc().parallelize(records, 1);
|
||||
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfig();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Test tagLocation without any entries in index
|
||||
JavaRDD<HoodieRecord> records1 = index.tagLocation(writeRecords, context(), hoodieTable);
|
||||
assertEquals(0, records1.filter(record -> record.isCurrentLocationKnown()).count());
|
||||
|
||||
// Insert records
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
JavaRDD<WriteStatus> writeStatues = writeClient.upsert(writeRecords, newCommitTime);
|
||||
assertNoWriteErrors(writeStatues.collect());
|
||||
writeClient.commit(newCommitTime, writeStatues);
|
||||
|
||||
// Now tagLocation for these records, hbaseIndex should tag them correctly
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
List<HoodieRecord> records2 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(numRecords, records2.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records2.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(numRecords, records2.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
|
||||
// Delete all records. This has to be done directly as deleting index entries
|
||||
// is not implemented via HoodieWriteClient
|
||||
JavaRDD<WriteStatus> deleteWriteStatues = writeStatues.map(w -> {
|
||||
WriteStatus newWriteStatus = new WriteStatus(true, 1.0);
|
||||
w.getWrittenRecords().forEach(r -> newWriteStatus.markSuccess(new HoodieRecord(r.getKey(), null), Option.empty()));
|
||||
assertEquals(w.getTotalRecords(), newWriteStatus.getTotalRecords());
|
||||
newWriteStatus.setStat(new HoodieWriteStat());
|
||||
return newWriteStatus;
|
||||
});
|
||||
JavaRDD<WriteStatus> deleteStatus = index.updateLocation(deleteWriteStatues, context(), hoodieTable);
|
||||
assertEquals(deleteStatus.count(), deleteWriteStatues.count());
|
||||
assertNoWriteErrors(deleteStatus.collect());
|
||||
|
||||
// Ensure no records can be tagged
|
||||
List<HoodieRecord> records3 = index.tagLocation(writeRecords, context(), hoodieTable).collect();
|
||||
assertEquals(0, records3.stream().filter(record -> record.isCurrentLocationKnown()).count());
|
||||
assertEquals(numRecords, records3.stream().map(record -> record.getKey().getRecordKey()).distinct().count());
|
||||
assertEquals(0, records3.stream().filter(record -> (record.getCurrentLocation() != null
|
||||
&& record.getCurrentLocation().getInstantTime().equals(newCommitTime))).distinct().count());
|
||||
}
|
||||
}
|
||||
|
||||
private WriteStatus getSampleWriteStatus(final int numInserts, final int numUpdateWrites) {
|
||||
final WriteStatus writeStatus = new WriteStatus(false, 0.1);
|
||||
HoodieWriteStat hoodieWriteStat = new HoodieWriteStat();
|
||||
hoodieWriteStat.setNumInserts(numInserts);
|
||||
hoodieWriteStat.setNumUpdateWrites(numUpdateWrites);
|
||||
writeStatus.setStat(hoodieWriteStat);
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder(100).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(int hbaseIndexBatchSize) {
|
||||
return getConfigBuilder(hbaseIndexBatchSize).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(int hbaseIndexBatchSize) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(1, 1).withDeleteParallelism(1)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.HBASE)
|
||||
.withHBaseIndexConfig(new HoodieHBaseIndexConfig.Builder()
|
||||
.hbaseZkPort(Integer.parseInt(hbaseConfig.get("hbase.zookeeper.property.clientPort")))
|
||||
.hbaseIndexPutBatchSizeAutoCompute(true)
|
||||
.hbaseZkZnodeParent(hbaseConfig.get("zookeeper.znode.parent", ""))
|
||||
.hbaseZkQuorum(hbaseConfig.get("hbase.zookeeper.quorum")).hbaseTableName(TABLE_NAME)
|
||||
.hbaseIndexGetBatchSize(hbaseIndexBatchSize).build())
|
||||
.build());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,40 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestHBaseIndexUsage {
|
||||
|
||||
@Test
|
||||
public void testFeatureSupport() {
|
||||
HoodieWriteConfig config = mock(HoodieWriteConfig.class);
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
assertTrue(index.canIndexLogFiles());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,64 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestHBasePutBatchSizeCalculator {
|
||||
|
||||
@Test
|
||||
public void testPutBatchSizeCalculation() {
|
||||
SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator batchSizeCalculator = new SparkHoodieHBaseIndex.HBasePutBatchSizeCalculator();
|
||||
|
||||
// All asserts cases below are derived out of the first
|
||||
// example below, with change in one parameter at a time.
|
||||
|
||||
int putBatchSize = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.1f);
|
||||
// Expected batchSize is 8 because in that case, total request sent in one second is below
|
||||
// 8 (batchSize) * 200 (parallelism) * 10 (maxReqsInOneSecond) * 10 (numRegionServers) * 0.1 (qpsFraction)) => 16000
|
||||
// We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 request
|
||||
// 1600 happens to be 10% of 16667 (maxQPSPerRegionServer) as expected.
|
||||
assertEquals(8, putBatchSize);
|
||||
|
||||
// Number of Region Servers are halved, total requests sent in a second are also halved, so batchSize is also halved
|
||||
int putBatchSize2 = batchSizeCalculator.getBatchSize(5, 16667, 1200, 200, 100, 0.1f);
|
||||
assertEquals(4, putBatchSize2);
|
||||
|
||||
// If the parallelism is halved, batchSize has to double
|
||||
int putBatchSize3 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 100, 100, 0.1f);
|
||||
assertEquals(16, putBatchSize3);
|
||||
|
||||
// If the parallelism is halved, batchSize has to double.
|
||||
// This time parallelism is driven by numTasks rather than numExecutors
|
||||
int putBatchSize4 = batchSizeCalculator.getBatchSize(10, 16667, 100, 200, 100, 0.1f);
|
||||
assertEquals(16, putBatchSize4);
|
||||
|
||||
// If sleepTimeMs is halved, batchSize has to halve
|
||||
int putBatchSize5 = batchSizeCalculator.getBatchSize(10, 16667, 1200, 200, 100, 0.05f);
|
||||
assertEquals(4, putBatchSize5);
|
||||
|
||||
// If maxQPSPerRegionServer is doubled, batchSize also doubles
|
||||
int putBatchSize6 = batchSizeCalculator.getBatchSize(10, 33334, 1200, 200, 100, 0.1f);
|
||||
assertEquals(16, putBatchSize6);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,96 @@
|
||||
/*
|
||||
* 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.index.hbase;
|
||||
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieHBaseIndexConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestHBaseQPSResourceAllocator {
|
||||
|
||||
@Test
|
||||
public void testsDefaultQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Option.empty());
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
DefaultHBaseQPSResourceAllocator.class.getName());
|
||||
assertEquals(config.getHbaseIndexQPSFraction(),
|
||||
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsExplicitDefaultQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Option.of(HoodieHBaseIndexConfig.DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
DefaultHBaseQPSResourceAllocator.class.getName());
|
||||
assertEquals(config.getHbaseIndexQPSFraction(),
|
||||
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testsInvalidQPSResourceAllocator() {
|
||||
HoodieWriteConfig config = getConfig(Option.of("InvalidResourceAllocatorClassName"));
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = index.createQPSResourceAllocator(config);
|
||||
assertEquals(hBaseIndexQPSResourceAllocator.getClass().getName(),
|
||||
DefaultHBaseQPSResourceAllocator.class.getName());
|
||||
assertEquals(config.getHbaseIndexQPSFraction(),
|
||||
hBaseIndexQPSResourceAllocator.acquireQPSResources(config.getHbaseIndexQPSFraction(), 100), 0.0f);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig(Option<String> resourceAllocatorClass) {
|
||||
HoodieHBaseIndexConfig hoodieHBaseIndexConfig = getConfigWithResourceAllocator(resourceAllocatorClass);
|
||||
return getConfigBuilder(hoodieHBaseIndexConfig).build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder(HoodieHBaseIndexConfig hoodieHBaseIndexConfig) {
|
||||
return HoodieWriteConfig.newBuilder().withPath("/foo").withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(1, 1)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withAutoCommit(false).withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1000 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table").withIndexConfig(HoodieIndexConfig.newBuilder()
|
||||
.withIndexType(HoodieIndex.IndexType.HBASE).withHBaseIndexConfig(hoodieHBaseIndexConfig).build());
|
||||
}
|
||||
|
||||
private HoodieHBaseIndexConfig getConfigWithResourceAllocator(Option<String> resourceAllocatorClass) {
|
||||
HoodieHBaseIndexConfig.Builder builder = new HoodieHBaseIndexConfig.Builder()
|
||||
.hbaseZkPort(0)
|
||||
.hbaseZkQuorum("localhost")
|
||||
.hbaseTableName("foobar")
|
||||
.hbaseIndexGetBatchSize(100);
|
||||
if (resourceAllocatorClass.isPresent()) {
|
||||
builder.withQPSResourceAllocatorType(resourceAllocatorClass.get());
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,165 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndexUtils;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieWriteableTestTable;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
|
||||
import static org.apache.hudi.common.testutils.Transformations.recordsToPartitionRecordsMap;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Tests {@link HoodieKeyLocationFetchHandle}.
|
||||
*/
|
||||
public class TestHoodieKeyLocationFetchHandle extends HoodieClientTestHarness {
|
||||
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestRecordFetcher");
|
||||
initPath();
|
||||
initTestDataGenerator();
|
||||
initFileSystem();
|
||||
initMetaClient();
|
||||
config = getConfigBuilder()
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder()
|
||||
.build()).build();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFetchHandle() throws Exception {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(makeNewCommitTime(), 100);
|
||||
Map<String, List<HoodieRecord>> partitionRecordsMap = recordsToPartitionRecordsMap(records);
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieWriteableTestTable testTable = HoodieWriteableTestTable.of(hoodieTable, AVRO_SCHEMA_WITH_METADATA_FIELDS);
|
||||
Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList =
|
||||
writeToParquetAndGetExpectedRecordLocations(partitionRecordsMap, testTable);
|
||||
|
||||
List<Tuple2<String, HoodieBaseFile>> partitionPathFileIdPairs = loadAllFilesForPartitions(new ArrayList<>(partitionRecordsMap.keySet()), context, hoodieTable);
|
||||
|
||||
for (Tuple2<String, HoodieBaseFile> entry : partitionPathFileIdPairs) {
|
||||
HoodieKeyLocationFetchHandle fetcherHandle = new HoodieKeyLocationFetchHandle(config, hoodieTable, Pair.of(entry._1, entry._2));
|
||||
Iterator<Tuple2<HoodieKey, HoodieRecordLocation>> result = fetcherHandle.locations();
|
||||
List<Tuple2<HoodieKey, HoodieRecordLocation>> actualList = new ArrayList<>();
|
||||
result.forEachRemaining(actualList::add);
|
||||
assertEquals(expectedList.get(new Tuple2<>(entry._1, entry._2.getFileId())), actualList);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> writeToParquetAndGetExpectedRecordLocations(
|
||||
Map<String, List<HoodieRecord>> partitionRecordsMap, HoodieWriteableTestTable testTable) throws Exception {
|
||||
Map<Tuple2<String, String>, List<Tuple2<HoodieKey, HoodieRecordLocation>>> expectedList = new HashMap<>();
|
||||
for (Map.Entry<String, List<HoodieRecord>> entry : partitionRecordsMap.entrySet()) {
|
||||
int totalRecordsPerPartition = entry.getValue().size();
|
||||
int totalSlices = 1;
|
||||
if (totalRecordsPerPartition > 5) {
|
||||
totalSlices = totalRecordsPerPartition / 3;
|
||||
}
|
||||
int recordsPerFileSlice = totalRecordsPerPartition / totalSlices;
|
||||
|
||||
List<List<HoodieRecord>> recordsForFileSlices = new ArrayList<>();
|
||||
recordsForFileSlices.add(new ArrayList<>());
|
||||
int index = 0;
|
||||
int count = 0;
|
||||
for (HoodieRecord record : entry.getValue()) {
|
||||
if (count < recordsPerFileSlice) {
|
||||
recordsForFileSlices.get(index).add(record);
|
||||
count++;
|
||||
} else {
|
||||
recordsForFileSlices.add(new ArrayList<>());
|
||||
index++;
|
||||
count = 0;
|
||||
}
|
||||
}
|
||||
|
||||
for (List<HoodieRecord> recordsPerSlice : recordsForFileSlices) {
|
||||
String instantTime = makeNewCommitTime();
|
||||
String fileId = testTable.addCommit(instantTime).withInserts(entry.getKey(), recordsPerSlice.toArray(new HoodieRecord[0]));
|
||||
Tuple2<String, String> fileIdInstantTimePair = new Tuple2<>(fileId, instantTime);
|
||||
List<Tuple2<HoodieKey, HoodieRecordLocation>> expectedEntries = new ArrayList<>();
|
||||
for (HoodieRecord record : recordsPerSlice) {
|
||||
expectedEntries.add(new Tuple2<>(record.getKey(), new HoodieRecordLocation(fileIdInstantTimePair._2, fileIdInstantTimePair._1)));
|
||||
}
|
||||
expectedList.put(new Tuple2<>(entry.getKey(), fileIdInstantTimePair._1), expectedEntries);
|
||||
}
|
||||
}
|
||||
return expectedList;
|
||||
}
|
||||
|
||||
private static List<Tuple2<String, HoodieBaseFile>> loadAllFilesForPartitions(List<String> partitions, HoodieEngineContext context,
|
||||
HoodieTable hoodieTable) {
|
||||
// Obtain the latest data files from all the partitions.
|
||||
List<Pair<String, HoodieBaseFile>> partitionPathFileIDList = HoodieIndexUtils.getLatestBaseFilesForAllPartitions(partitions, context, hoodieTable);
|
||||
return partitionPathFileIDList.stream()
|
||||
.map(pf -> new Tuple2<>(pf.getKey(), pf.getValue())).collect(toList());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
}
|
||||
@@ -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.io;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public class TestHoodieMergeHandle extends HoodieClientTestHarness {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initPath();
|
||||
initFileSystem();
|
||||
initTestDataGenerator();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertsForMultipleRecordsInSameFile() throws Exception {
|
||||
// Create records in a single partition
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {partitionPath});
|
||||
|
||||
// Build a write config with bulkinsertparallelism set
|
||||
HoodieWriteConfig cfg = getConfigBuilder().build();
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
|
||||
FileSystem fs = FSUtils.getFs(basePath, hadoopConf);
|
||||
|
||||
/**
|
||||
* Write 1 (only inserts) This will do a bulk insert of 44 records of which there are 2 records repeated 21 times
|
||||
* each. id1 (21 records), id2 (21 records), id3, id4
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 4);
|
||||
HoodieRecord record1 = records.get(0);
|
||||
HoodieRecord record2 = records.get(1);
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
for (int i = 0; i < 20; i++) {
|
||||
HoodieRecord dup = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
records.add(dup);
|
||||
}
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there is a commit
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(),
|
||||
"Expecting a single commit.");
|
||||
assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 001");
|
||||
assertEquals(records.size(),
|
||||
HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
"Must contain 44 records");
|
||||
|
||||
/**
|
||||
* Write 2 (insert) This will do a bulk insert of 1 record with the same row_key as record1 in the previous insert
|
||||
* - id1. At this point, we will have 2 files with the row_keys as shown here - File 1 - id1 (21 records), id2 (21
|
||||
* records), id3, id4 File 2 - id1
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
// Do 1 more bulk insert with the same dup record1
|
||||
List<HoodieRecord> newRecords = new ArrayList<>();
|
||||
HoodieRecord sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
newRecords.add(sameAsRecord1);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are 2 commits
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals(2, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting two commits.");
|
||||
assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 002");
|
||||
Dataset<Row> dataSet = getRecords();
|
||||
assertEquals(45, dataSet.count(), "Must contain 45 records");
|
||||
|
||||
/**
|
||||
* Write 3 (insert) This will bulk insert 2 new completely new records. At this point, we will have 2 files with
|
||||
* the row_keys as shown here - File 1 - id1 (21 records), id2 (21 records), id3, id4 File 2 - id1 File 3 - id5,
|
||||
* id6
|
||||
*/
|
||||
newCommitTime = "003";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
newRecords = dataGen.generateInserts(newCommitTime, 2);
|
||||
writeRecords = jsc.parallelize(newRecords, 1);
|
||||
statuses = client.bulkInsert(writeRecords, newCommitTime).collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify that there are now 3 commits
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals(3, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting three commits.");
|
||||
assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 003");
|
||||
dataSet = getRecords();
|
||||
assertEquals(47, dataSet.count(), "Must contain 47 records");
|
||||
|
||||
/**
|
||||
* Write 4 (updates) This will generate 2 upsert records with id1 and id2. The rider and driver names in the
|
||||
* update records will be rider-004 and driver-004. After the upsert is complete, all the records with id1 in File
|
||||
* 1 and File 2 must be updated, all the records with id2 in File 2 must also be updated. Also, none of the other
|
||||
* records in File 1, File 2 and File 3 must be updated.
|
||||
*/
|
||||
newCommitTime = "004";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> updateRecords = new ArrayList<>();
|
||||
|
||||
// This exists in 001 and 002 and should be updated in both
|
||||
sameAsRecord1 = dataGen.generateUpdateRecord(record1.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord1);
|
||||
|
||||
// This exists in 001 and should be updated
|
||||
HoodieRecord sameAsRecord2 = dataGen.generateUpdateRecord(record2.getKey(), newCommitTime);
|
||||
updateRecords.add(sameAsRecord2);
|
||||
JavaRDD<HoodieRecord> updateRecordsRDD = jsc.parallelize(updateRecords, 1);
|
||||
statuses = client.upsert(updateRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// Verify there are no errors
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// verify there are now 4 commits
|
||||
timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
assertEquals(4, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting four commits.");
|
||||
assertEquals(timeline.lastInstant().get().getTimestamp(), newCommitTime, "Latest commit should be 004");
|
||||
|
||||
// Check the entire dataset has 47 records still
|
||||
dataSet = getRecords();
|
||||
assertEquals(47, dataSet.count(), "Must contain 47 records");
|
||||
Row[] rows = (Row[]) dataSet.collect();
|
||||
int record1Count = 0;
|
||||
int record2Count = 0;
|
||||
for (Row row : rows) {
|
||||
if (row.getAs("_hoodie_record_key").equals(record1.getKey().getRecordKey())) {
|
||||
record1Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else if (row.getAs("_hoodie_record_key").equals(record2.getKey().getRecordKey())) {
|
||||
record2Count++;
|
||||
// assert each duplicate record is updated
|
||||
assertEquals(row.getAs("rider"), "rider-004");
|
||||
assertEquals(row.getAs("driver"), "driver-004");
|
||||
} else {
|
||||
assertNotEquals(row.getAs("rider"), "rider-004");
|
||||
assertNotEquals(row.getAs("driver"), "rider-004");
|
||||
}
|
||||
}
|
||||
// Assert that id1 record count which has been updated to rider-004 and driver-004 is 22, which is the total
|
||||
// number of records with row_key id1
|
||||
assertEquals(22, record1Count);
|
||||
|
||||
// Assert that id2 record count which has been updated to rider-004 and driver-004 is 21, which is the total
|
||||
// number of records with row_key id2
|
||||
assertEquals(21, record2Count);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHoodieMergeHandleWriteStatMetrics() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfigBuilder().build();
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
List<WriteStatus> statuses = writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be inserts into new parquet
|
||||
assertTrue(statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() != HoodieWriteStat.NULL_COMMIT).count() > 0);
|
||||
// Num writes should be equal to the number of records inserted
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of records updated
|
||||
assertEquals(0,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get());
|
||||
|
||||
// Update all the 100 records
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
statuses = writeClient.upsert(updatedRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet
|
||||
assertEquals(0,
|
||||
statuses.stream().filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count());
|
||||
// Num writes should be equal to the number of records inserted
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of records updated
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
assertEquals(0,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get());
|
||||
|
||||
newCommitTime = "102";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> allRecords = dataGen.generateInserts(newCommitTime, 100);
|
||||
allRecords.addAll(updatedRecords);
|
||||
JavaRDD<HoodieRecord> allRecordsRDD = jsc.parallelize(allRecords, 1);
|
||||
statuses = writeClient.upsert(allRecordsRDD, newCommitTime).collect();
|
||||
|
||||
// All records should be upserts into existing parquet (with inserts as updates small file handled)
|
||||
assertEquals(0, (long) statuses.stream()
|
||||
.filter(status -> status.getStat().getPrevCommit() == HoodieWriteStat.NULL_COMMIT).count());
|
||||
// Num writes should be equal to the total number of records written
|
||||
assertEquals(200,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of records updated (including inserts converted as updates)
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumUpdateWrites()).reduce((a, b) -> a + b).get());
|
||||
// Num update writes should be equal to the number of insert records converted to updates as part of small file
|
||||
// handling
|
||||
assertEquals(100,
|
||||
(long) statuses.stream().map(status -> status.getStat().getNumInserts()).reduce((a, b) -> a + b).get());
|
||||
// Verify all records have location set
|
||||
statuses.forEach(writeStatus -> {
|
||||
writeStatus.getWrittenRecords().forEach(r -> {
|
||||
// Ensure New Location is set
|
||||
assertTrue(r.getNewLocation().isPresent());
|
||||
});
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
private Dataset<Row> getRecords() {
|
||||
// Check the entire dataset has 8 records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = Paths.get(basePath, dataGen.getPartitionPaths()[i], "*").toString();
|
||||
}
|
||||
Dataset<Row> dataSet = HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths);
|
||||
return dataSet;
|
||||
}
|
||||
|
||||
HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withDeleteParallelism(2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withBulkInsertParallelism(2).withWriteStatusClass(TestWriteStatus.class);
|
||||
}
|
||||
|
||||
/**
|
||||
* Overridden so that we can capture and inspect all success records.
|
||||
*/
|
||||
public static class TestWriteStatus extends WriteStatus {
|
||||
|
||||
public TestWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
// Track Success Records
|
||||
super(true, failureFraction);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,228 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hudi.client.HoodieInternalWriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieRowCreateHandle}.
|
||||
*/
|
||||
@SuppressWarnings("checkstyle:LineLength")
|
||||
public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieRowCreateHandle");
|
||||
initPath();
|
||||
initFileSystem();
|
||||
initTestDataGenerator();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRowCreateHandle() throws IOException {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
List<String> fileNames = new ArrayList<>();
|
||||
List<String> fileAbsPaths = new ArrayList<>();
|
||||
|
||||
Dataset<Row> totalInputRows = null;
|
||||
// one round per partition
|
||||
for (int i = 0; i < 5; i++) {
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[i % 3];
|
||||
|
||||
// init some args
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String instantTime = "000";
|
||||
|
||||
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
// Generate inputs
|
||||
Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
|
||||
if (totalInputRows == null) {
|
||||
totalInputRows = inputRows;
|
||||
} else {
|
||||
totalInputRows = totalInputRows.union(inputRows);
|
||||
}
|
||||
|
||||
// issue writes
|
||||
HoodieInternalWriteStatus writeStatus = writeAndGetWriteStatus(inputRows, handle);
|
||||
|
||||
fileAbsPaths.add(basePath + "/" + writeStatus.getStat().getPath());
|
||||
fileNames.add(handle.getFileName());
|
||||
// verify output
|
||||
assertOutput(writeStatus, size, fileId, partitionPath, instantTime, totalInputRows, fileNames, fileAbsPaths);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Issue some corrupted or wrong schematized InternalRow after few valid InternalRows so that global error is thrown. write batch 1 of valid records write batch 2 of invalid records Global Error
|
||||
* should be thrown.
|
||||
*/
|
||||
@Test
|
||||
public void testGlobalFailure() throws IOException {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
|
||||
|
||||
// init some args
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
String instantTime = "000";
|
||||
|
||||
HoodieRowCreateHandle handle = new HoodieRowCreateHandle(table, cfg, partitionPath, fileId, instantTime, RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
int size = 10 + RANDOM.nextInt(1000);
|
||||
int totalFailures = 5;
|
||||
// Generate first batch of valid rows
|
||||
Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false);
|
||||
List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
|
||||
|
||||
// generate some failures rows
|
||||
for (int i = 0; i < totalFailures; i++) {
|
||||
internalRows.add(SparkDatasetTestUtils.getInternalRowWithError(partitionPath));
|
||||
}
|
||||
|
||||
// generate 2nd batch of valid rows
|
||||
Dataset<Row> inputRows2 = SparkDatasetTestUtils.getRandomRows(sqlContext, size / 2, partitionPath, false);
|
||||
internalRows.addAll(SparkDatasetTestUtils.toInternalRows(inputRows2, SparkDatasetTestUtils.ENCODER));
|
||||
|
||||
// issue writes
|
||||
try {
|
||||
for (InternalRow internalRow : internalRows) {
|
||||
handle.write(internalRow);
|
||||
}
|
||||
fail("Should have failed");
|
||||
} catch (Throwable e) {
|
||||
// expected
|
||||
}
|
||||
// close the create handle
|
||||
HoodieInternalWriteStatus writeStatus = handle.close();
|
||||
|
||||
List<String> fileNames = new ArrayList<>();
|
||||
fileNames.add(handle.getFileName());
|
||||
// verify write status
|
||||
assertNotNull(writeStatus.getGlobalError());
|
||||
assertTrue(writeStatus.getGlobalError().getMessage().contains("java.lang.String cannot be cast to org.apache.spark.unsafe.types.UTF8String"));
|
||||
assertEquals(writeStatus.getFileId(), fileId);
|
||||
assertEquals(writeStatus.getPartitionPath(), partitionPath);
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(basePath + "/" + partitionPath);
|
||||
// passing only first batch of inputRows since after first batch global error would have been thrown
|
||||
assertRows(inputRows, result, instantTime, fileNames);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInstantiationFailure() throws IOException {
|
||||
// init config and table
|
||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withPath("/dummypath/abc/").build();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
|
||||
try {
|
||||
new HoodieRowCreateHandle(table, cfg, " def", UUID.randomUUID().toString(), "001", RANDOM.nextInt(100000), RANDOM.nextLong(), RANDOM.nextLong(), SparkDatasetTestUtils.STRUCT_TYPE);
|
||||
fail("Should have thrown exception");
|
||||
} catch (HoodieInsertException ioe) {
|
||||
// expected
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieInternalWriteStatus writeAndGetWriteStatus(Dataset<Row> inputRows, HoodieRowCreateHandle handle) throws IOException {
|
||||
List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
|
||||
// issue writes
|
||||
for (InternalRow internalRow : internalRows) {
|
||||
handle.write(internalRow);
|
||||
}
|
||||
// close the create handle
|
||||
return handle.close();
|
||||
}
|
||||
|
||||
private void assertOutput(HoodieInternalWriteStatus writeStatus, int size, String fileId, String partitionPath, String instantTime, Dataset<Row> inputRows, List<String> filenames,
|
||||
List<String> fileAbsPaths) {
|
||||
assertEquals(writeStatus.getPartitionPath(), partitionPath);
|
||||
assertEquals(writeStatus.getTotalRecords(), size);
|
||||
assertEquals(writeStatus.getFailedRowsSize(), 0);
|
||||
assertEquals(writeStatus.getTotalErrorRecords(), 0);
|
||||
assertFalse(writeStatus.hasErrors());
|
||||
assertNull(writeStatus.getGlobalError());
|
||||
assertEquals(writeStatus.getFileId(), fileId);
|
||||
HoodieWriteStat writeStat = writeStatus.getStat();
|
||||
assertEquals(size, writeStat.getNumInserts());
|
||||
assertEquals(size, writeStat.getNumWrites());
|
||||
assertEquals(fileId, writeStat.getFileId());
|
||||
assertEquals(partitionPath, writeStat.getPartitionPath());
|
||||
assertEquals(0, writeStat.getNumDeletes());
|
||||
assertEquals(0, writeStat.getNumUpdateWrites());
|
||||
assertEquals(0, writeStat.getTotalWriteErrors());
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(fileAbsPaths.toArray(new String[0]));
|
||||
assertRows(inputRows, result, instantTime, filenames);
|
||||
}
|
||||
|
||||
private void assertRows(Dataset<Row> expectedRows, Dataset<Row> actualRows, String instantTime, List<String> filenames) {
|
||||
// verify 3 meta fields that are filled in within create handle
|
||||
actualRows.collectAsList().forEach(entry -> {
|
||||
assertEquals(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD)).toString(), instantTime);
|
||||
assertTrue(filenames.contains(entry.get(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.FILENAME_METADATA_FIELD)).toString()));
|
||||
assertFalse(entry.isNullAt(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD)));
|
||||
});
|
||||
|
||||
// after trimming 2 of the meta fields, rest of the fields should match
|
||||
Dataset<Row> trimmedExpected = expectedRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
|
||||
Dataset<Row> trimmedActual = actualRows.drop(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD, HoodieRecord.FILENAME_METADATA_FIELD);
|
||||
assertEquals(0, trimmedActual.except(trimmedExpected).count());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,494 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.HoodieTimelineArchiveLog;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private HoodieWrapperFileSystem wrapperFs;
|
||||
|
||||
@BeforeEach
|
||||
public void init() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
initMetaClient();
|
||||
hadoopConf = context.getHadoopConf().get();
|
||||
metaClient.getFs().mkdirs(new Path(basePath));
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath);
|
||||
wrapperFs = metaClient.getFs();
|
||||
hadoopConf.addResource(wrapperFs.getConf());
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void clean() throws IOException {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveEmptyTable() throws IOException {
|
||||
HoodieWriteConfig cfg =
|
||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).forTable("test-trip-table").build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveTableWithArchival() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 4).build())
|
||||
.forTable("test-trip-table").build();
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
|
||||
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "100", wrapperFs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "101", wrapperFs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "102", wrapperFs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", wrapperFs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", wrapperFs.getConf());
|
||||
HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", wrapperFs.getConf());
|
||||
HoodieTestUtils.createPendingCleanFiles(metaClient, "106", "107");
|
||||
|
||||
// reload the timeline and get all the commmits before archive
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
|
||||
|
||||
assertEquals(12, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
|
||||
// verify in-flight instants before archive
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
assertTrue(archiveLog.archiveIfRequired(context));
|
||||
|
||||
// reload the timeline and remove the remaining commits
|
||||
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
|
||||
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
|
||||
|
||||
// Check compaction instants
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals(4, instants.size(), "Should delete all compaction instants < 104");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")),
|
||||
"Requested Compaction must be absent for 100");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")),
|
||||
"Inflight Compaction must be absent for 100");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
|
||||
"Requested Compaction must be absent for 101");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")),
|
||||
"Inflight Compaction must be absent for 101");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")),
|
||||
"Requested Compaction must be absent for 102");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")),
|
||||
"Inflight Compaction must be absent for 102");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")),
|
||||
"Requested Compaction must be absent for 103");
|
||||
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")),
|
||||
"Inflight Compaction must be absent for 103");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")),
|
||||
"Requested Compaction must be present for 104");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104")),
|
||||
"Inflight Compaction must be present for 104");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105")),
|
||||
"Requested Compaction must be present for 105");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105")),
|
||||
"Inflight Compaction must be present for 105");
|
||||
|
||||
// read the file
|
||||
HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient);
|
||||
assertEquals(24, archivedTimeline.countInstants(),
|
||||
"Total archived records and total read records are the same count");
|
||||
|
||||
//make sure the archived commits are the same as the (originalcommits - commitsleft)
|
||||
Set<String> readCommits =
|
||||
archivedTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
|
||||
assertEquals(originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits,
|
||||
"Read commits map should match the originalCommits - commitsLoadedFromArchival");
|
||||
|
||||
// verify in-flight instants after archive
|
||||
verifyInflightInstants(metaClient, 2);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveTableWithReplacedFiles() throws Exception {
|
||||
HoodieTestUtils.init(hadoopConf, basePath);
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||
.build();
|
||||
|
||||
int numCommits = 4;
|
||||
int commitInstant = 100;
|
||||
for (int i = 0; i < numCommits; i++) {
|
||||
createReplaceMetadata(commitInstant);
|
||||
commitInstant += 100;
|
||||
}
|
||||
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
|
||||
FileStatus[] allFiles = metaClient.getFs().listStatus(new Path(basePath + "/" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH));
|
||||
Set<String> allFileIds = Arrays.stream(allFiles).map(fs -> FSUtils.getFileIdFromFilePath(fs.getPath())).collect(Collectors.toSet());
|
||||
|
||||
// verify 100-1,200-1 are deleted by archival
|
||||
assertFalse(allFileIds.contains("file-100-1"));
|
||||
assertFalse(allFileIds.contains("file-200-1"));
|
||||
assertTrue(allFileIds.contains("file-100-2"));
|
||||
assertTrue(allFileIds.contains("file-200-2"));
|
||||
assertTrue(allFileIds.contains("file-300-1"));
|
||||
assertTrue(allFileIds.contains("file-400-1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveTableWithNoArchival() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
|
||||
// Requested Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf());
|
||||
// Inflight Compaction
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5");
|
||||
|
||||
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
|
||||
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
|
||||
assertEquals(8, instants.size(), "Should not delete any aux compaction files when maxCommitsToKeep is 5");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")),
|
||||
"Requested Compaction must be present for 100");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")),
|
||||
"Inflight Compaction must be present for 100");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
|
||||
"Requested Compaction must be present for 101");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")),
|
||||
"Inflight Compaction must be present for 101");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")),
|
||||
"Requested Compaction must be present for 102");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")),
|
||||
"Inflight Compaction must be present for 102");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")),
|
||||
"Requested Compaction must be present for 103");
|
||||
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")),
|
||||
"Inflight Compaction must be present for 103");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveCommitSafety() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe");
|
||||
assertTrue(timeline.containsOrBeforeTimelineStarts("101"), "Archived commits should always be safe");
|
||||
assertTrue(timeline.containsOrBeforeTimelineStarts("102"), "Archived commits should always be safe");
|
||||
assertTrue(timeline.containsOrBeforeTimelineStarts("103"), "Archived commits should always be safe");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveCommitSavepointNoHole() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createSavepointFile(basePath, "101", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
assertTrue(archiveLog.archiveIfRequired(context));
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
|
||||
assertEquals(5, timeline.countInstants(),
|
||||
"Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")),
|
||||
"Archived commits should always be safe");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")),
|
||||
"Archived commits should always be safe");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")),
|
||||
"Archived commits should always be safe");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveCommitCompactionNoHole() throws IOException {
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
|
||||
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf());
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline();
|
||||
assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match");
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
timeline = metaClient.getActiveTimeline().reload().getCommitsAndCompactionTimeline();
|
||||
assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")),
|
||||
"Instants before oldest pending compaction can be removed");
|
||||
assertEquals(7, timeline.countInstants(),
|
||||
"Since we have a pending compaction at 101, we should never archive any commit "
|
||||
+ "after 101 (we only archive 100)");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
|
||||
"Requested Compaction must still be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")),
|
||||
"Instants greater than oldest pending compaction must be present");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testArchiveCommitTimeline() throws IOException {
|
||||
HoodieWriteConfig cfg =
|
||||
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2).forTable("test-trip-table")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "1", wrapperFs.getConf());
|
||||
HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "2", wrapperFs.getConf());
|
||||
Path markerPath = new Path(metaClient.getMarkerFolderPath("2"));
|
||||
wrapperFs.mkdirs(markerPath);
|
||||
HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2");
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "3", wrapperFs.getConf());
|
||||
HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3");
|
||||
|
||||
//add 2 more instants to pass filter criteria set in compaction config above
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "4", wrapperFs.getConf());
|
||||
HoodieTestDataGenerator.createCommitFile(basePath, "5", wrapperFs.getConf());
|
||||
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
boolean result = archiveLog.archiveIfRequired(context);
|
||||
assertTrue(result);
|
||||
HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline();
|
||||
List<HoodieInstant> archivedInstants = Arrays.asList(instant1, instant2, instant3);
|
||||
assertEquals(new HashSet<>(archivedInstants), archivedTimeline.getInstants().collect(Collectors.toSet()));
|
||||
assertFalse(wrapperFs.exists(markerPath));
|
||||
}
|
||||
|
||||
private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) {
|
||||
HoodieTimeline timeline = metaClient.getActiveTimeline().reload()
|
||||
.getTimelineOfActions(Collections.singleton(HoodieTimeline.CLEAN_ACTION)).filterInflights();
|
||||
assertEquals(expectedTotalInstants, timeline.countInstants(),
|
||||
"Loaded inflight clean actions and the count should match");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConvertCommitMetadata() {
|
||||
HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
|
||||
hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
|
||||
|
||||
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-commitMetadata-converter")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build())
|
||||
.build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
|
||||
|
||||
org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata);
|
||||
assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
|
||||
}
|
||||
|
||||
private void createReplaceMetadata(int commitInstant) throws Exception {
|
||||
String commitTime = "" + commitInstant;
|
||||
String fileId1 = "file-" + commitInstant + "-1";
|
||||
String fileId2 = "file-" + commitInstant + "-2";
|
||||
|
||||
// create replace instant to mark fileId1 as deleted
|
||||
HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata();
|
||||
replaceMetadata.addReplaceFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1);
|
||||
replaceMetadata.setOperationType(WriteOperationType.INSERT_OVERWRITE);
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient);
|
||||
testTable.addReplaceCommit(commitTime, replaceMetadata);
|
||||
testTable.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* 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.io;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.config.HoodieMemoryConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_MERGE_PROP;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestSparkIOUtils {
|
||||
@TempDir
|
||||
public java.nio.file.Path basePath;
|
||||
|
||||
private final SparkTaskContextSupplier contextSupplier = new SparkTaskContextSupplier();
|
||||
|
||||
@Test
|
||||
public void testMaxMemoryPerPartitionMergeWithMaxSizeDefined() {
|
||||
String path = basePath.toString();
|
||||
|
||||
long mergeMaxSize = 1000;
|
||||
long compactionMaxSize = 1000;
|
||||
|
||||
HoodieMemoryConfig memoryConfig = HoodieMemoryConfig.newBuilder().withMaxMemoryMaxSize(mergeMaxSize, compactionMaxSize).build();
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).withMemoryConfig(memoryConfig).build();
|
||||
|
||||
assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
|
||||
assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMaxMemoryPerPartitionMergeInDefault() {
|
||||
String path = basePath.toString();
|
||||
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(path).build();
|
||||
|
||||
String compactionFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
long compactionMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, compactionFraction);
|
||||
|
||||
String mergeFraction = config.getProps().getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
|
||||
long mergeMaxSize = IOUtils.getMaxMemoryAllowedForMerge(contextSupplier, mergeFraction);
|
||||
|
||||
assertEquals(mergeMaxSize, IOUtils.getMaxMemoryPerPartitionMerge(contextSupplier, config.getProps()));
|
||||
assertEquals(compactionMaxSize, IOUtils.getMaxMemoryPerCompaction(contextSupplier, config.getProps()));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Tests for {@link HoodieFileWriterFactory}.
|
||||
*/
|
||||
public class TestHoodieFileWriterFactory extends HoodieClientTestBase {
|
||||
|
||||
@Test
|
||||
public void testGetFileWriter() throws IOException {
|
||||
// parquet file format.
|
||||
final String instantTime = "100";
|
||||
final Path parquetPath = new Path(basePath + "/partition/path/f1_1-0-1_000.parquet");
|
||||
final HoodieWriteConfig cfg = getConfig();
|
||||
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
HoodieFileWriter<IndexedRecord> parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
assertTrue(parquetWriter instanceof HoodieParquetWriter);
|
||||
|
||||
final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile");
|
||||
HoodieFileWriter<IndexedRecord> hfileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
|
||||
hfilePath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
assertTrue(hfileWriter instanceof HoodieHFileWriter);
|
||||
|
||||
// other file format exception.
|
||||
final Path logPath = new Path(basePath + "/partition/path/f.b51192a8-574b-4a85-b246-bcfec03ac8bf_100.log.2_1-0-1");
|
||||
final Throwable thrown = assertThrows(UnsupportedOperationException.class, () -> {
|
||||
HoodieFileWriter<IndexedRecord> logWriter = HoodieFileWriterFactory.getFileWriter(instantTime, logPath,
|
||||
table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
}, "should fail since log storage writer is not supported yet.");
|
||||
assertTrue(thrown.getMessage().contains("format not supported yet."));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,113 @@
|
||||
/*
|
||||
* 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.io.storage;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.testutils.SparkDatasetTestUtils;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
/**
|
||||
* Unit tests {@link HoodieInternalRowParquetWriter}.
|
||||
*/
|
||||
public class TestHoodieInternalRowParquetWriter extends HoodieClientTestHarness {
|
||||
|
||||
private static final Random RANDOM = new Random();
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts("TestHoodieInternalRowParquetWriter");
|
||||
initPath();
|
||||
initFileSystem();
|
||||
initTestDataGenerator();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void endToEndTest() throws IOException {
|
||||
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).build();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
// init write support and parquet config
|
||||
HoodieRowParquetWriteSupport writeSupport = getWriteSupport(cfg, hadoopConf);
|
||||
HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
|
||||
CompressionCodecName.SNAPPY, cfg.getParquetBlockSize(), cfg.getParquetPageSize(), cfg.getParquetMaxFileSize(),
|
||||
writeSupport.getHadoopConf(), cfg.getParquetCompressionRatio());
|
||||
|
||||
// prepare path
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
Path filePath = new Path(basePath + "/" + fileId);
|
||||
String partitionPath = HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
metaClient.getFs().mkdirs(new Path(basePath));
|
||||
|
||||
// init writer
|
||||
HoodieInternalRowParquetWriter writer = new HoodieInternalRowParquetWriter(filePath, parquetConfig);
|
||||
|
||||
// generate input
|
||||
int size = 10 + RANDOM.nextInt(100);
|
||||
// Generate inputs
|
||||
Dataset<Row> inputRows = SparkDatasetTestUtils.getRandomRows(sqlContext, size, partitionPath, false);
|
||||
List<InternalRow> internalRows = SparkDatasetTestUtils.toInternalRows(inputRows, SparkDatasetTestUtils.ENCODER);
|
||||
|
||||
// issue writes
|
||||
for (InternalRow internalRow : internalRows) {
|
||||
writer.write(internalRow);
|
||||
}
|
||||
|
||||
// close the writer
|
||||
writer.close();
|
||||
|
||||
// verify rows
|
||||
Dataset<Row> result = sqlContext.read().parquet(basePath);
|
||||
assertEquals(0, inputRows.except(result).count());
|
||||
}
|
||||
}
|
||||
|
||||
private HoodieRowParquetWriteSupport getWriteSupport(HoodieWriteConfig writeConfig, Configuration hadoopConf) {
|
||||
BloomFilter filter = BloomFilterFactory.createBloomFilter(
|
||||
writeConfig.getBloomFilterNumEntries(),
|
||||
writeConfig.getBloomFilterFPP(),
|
||||
writeConfig.getDynamicBloomFilterMaxNumEntries(),
|
||||
writeConfig.getBloomFilterType());
|
||||
return new HoodieRowParquetWriteSupport(hadoopConf, SparkDatasetTestUtils.STRUCT_TYPE, filter);
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,168 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.common.fs.ConsistencyGuard;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FailSafeConsistencyGuard;
|
||||
import org.apache.hudi.common.fs.OptimisticConsistencyGuard;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
|
||||
/**
|
||||
* Unit tests {@link ConsistencyGuard}s.
|
||||
*/
|
||||
public class TestConsistencyGuard extends HoodieClientTestHarness {
|
||||
|
||||
// multiple parameters, uses Collection<Object[]>
|
||||
public static List<Arguments> consistencyGuardType() {
|
||||
return Arrays.asList(
|
||||
Arguments.of(FailSafeConsistencyGuard.class.getName()),
|
||||
Arguments.of(OptimisticConsistencyGuard.class.getName())
|
||||
);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setup() {
|
||||
initPath();
|
||||
initFileSystemWithDefaultConfiguration();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("consistencyGuardType")
|
||||
public void testCheckPassingAppearAndDisAppear(String consistencyGuardType) throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f2");
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f3");
|
||||
|
||||
ConsistencyGuardConfig config = getConsistencyGuardConfig(1, 1000, 1000);
|
||||
ConsistencyGuard passing = consistencyGuardType.equals(FailSafeConsistencyGuard.class.getName())
|
||||
? new FailSafeConsistencyGuard(fs, config) : new OptimisticConsistencyGuard(fs, config);
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
|
||||
fs.delete(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"), false);
|
||||
fs.delete(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"), false);
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-1_000.parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingAppearFailSafe() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingAppearTimedWait() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesAppear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-2_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingAppearsFailSafe() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingAppearsTimedWait() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileAppears(new Path(basePath + "/partition/path/f1_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingDisappearFailSafe() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingDisappearTimedWait() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillAllFilesDisappear(basePath + "/partition/path", Arrays
|
||||
.asList(basePath + "/partition/path/f1_1-0-1_000.parquet", basePath + "/partition/path/f2_1-0-2_000.parquet"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingDisappearsFailSafe() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new FailSafeConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
assertThrows(TimeoutException.class, () -> {
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCheckFailingDisappearsTimedWait() throws Exception {
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
FileCreateUtils.createBaseFile(basePath, "partition/path", "000", "f1");
|
||||
ConsistencyGuard passing = new OptimisticConsistencyGuard(fs, getConsistencyGuardConfig());
|
||||
passing.waitTillFileDisappears(new Path(basePath + "/partition/path/f1_1-0-1_000.parquet"));
|
||||
}
|
||||
|
||||
private ConsistencyGuardConfig getConsistencyGuardConfig() {
|
||||
return getConsistencyGuardConfig(3, 10, 10);
|
||||
}
|
||||
|
||||
private ConsistencyGuardConfig getConsistencyGuardConfig(int maxChecks, int initalSleep, int maxSleep) {
|
||||
return ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true)
|
||||
.withInitialConsistencyCheckIntervalMs(initalSleep).withMaxConsistencyCheckIntervalMs(maxSleep)
|
||||
.withMaxConsistencyChecks(maxChecks).build();
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -0,0 +1,162 @@
|
||||
/*
|
||||
* 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.table;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.testutils.FileSystemTestUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestMarkerFiles extends HoodieCommonTestHarness {
|
||||
|
||||
private MarkerFiles markerFiles;
|
||||
private FileSystem fs;
|
||||
private Path markerFolderPath;
|
||||
private JavaSparkContext jsc;
|
||||
private HoodieSparkEngineContext context;
|
||||
|
||||
@BeforeEach
|
||||
public void setup() throws IOException {
|
||||
initPath();
|
||||
initMetaClient();
|
||||
this.jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(TestMarkerFiles.class.getName()));
|
||||
this.context = new HoodieSparkEngineContext(jsc);
|
||||
this.fs = FSUtils.getFs(metaClient.getBasePath(), metaClient.getHadoopConf());
|
||||
this.markerFolderPath = new Path(metaClient.getMarkerFolderPath("000"));
|
||||
this.markerFiles = new MarkerFiles(fs, metaClient.getBasePath(), markerFolderPath.toString(), "000");
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void cleanup() {
|
||||
jsc.stop();
|
||||
context = null;
|
||||
}
|
||||
|
||||
private void createSomeMarkerFiles() {
|
||||
markerFiles.create("2020/06/01", "file1", IOType.MERGE);
|
||||
markerFiles.create("2020/06/02", "file2", IOType.APPEND);
|
||||
markerFiles.create("2020/06/03", "file3", IOType.CREATE);
|
||||
}
|
||||
|
||||
private void createInvalidFile(String partitionPath, String invalidFileName) {
|
||||
Path path = FSUtils.getPartitionPath(markerFolderPath.toString(), partitionPath);
|
||||
Path invalidFilePath = new Path(path, invalidFileName);
|
||||
try {
|
||||
fs.create(invalidFilePath, false).close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Failed to create invalid file " + invalidFilePath, e);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreation() throws Exception {
|
||||
// when
|
||||
createSomeMarkerFiles();
|
||||
|
||||
// then
|
||||
assertTrue(fs.exists(markerFolderPath));
|
||||
List<FileStatus> markerFiles = FileSystemTestUtils.listRecursive(fs, markerFolderPath)
|
||||
.stream().filter(status -> status.getPath().getName().contains(".marker"))
|
||||
.sorted().collect(Collectors.toList());
|
||||
assertEquals(3, markerFiles.size());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||
"file:" + markerFolderPath.toString() + "/2020/06/01/file1.marker.MERGE",
|
||||
"file:" + markerFolderPath.toString() + "/2020/06/02/file2.marker.APPEND",
|
||||
"file:" + markerFolderPath.toString() + "/2020/06/03/file3.marker.CREATE"),
|
||||
markerFiles.stream().map(m -> m.getPath().toString()).collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeletionWhenMarkerDirExists() throws IOException {
|
||||
//when
|
||||
markerFiles.create("2020/06/01", "file1", IOType.MERGE);
|
||||
|
||||
// then
|
||||
assertTrue(markerFiles.doesMarkerDirExist());
|
||||
assertTrue(markerFiles.deleteMarkerDir(context, 2));
|
||||
assertFalse(markerFiles.doesMarkerDirExist());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeletionWhenMarkerDirNotExists() throws IOException {
|
||||
// then
|
||||
assertFalse(markerFiles.doesMarkerDirExist());
|
||||
assertFalse(markerFiles.deleteMarkerDir(context, 2));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDataPathsWhenCreatingOrMerging() throws IOException {
|
||||
// add markfiles
|
||||
createSomeMarkerFiles();
|
||||
// add invalid file
|
||||
createInvalidFile("2020/06/01", "invalid_file3");
|
||||
int fileSize = FileSystemTestUtils.listRecursive(fs, markerFolderPath).size();
|
||||
assertEquals(fileSize,4);
|
||||
|
||||
// then
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(
|
||||
"2020/06/01/file1", "2020/06/03/file3"),
|
||||
markerFiles.createdAndMergedDataPaths(context, 2).stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAllMarkerPaths() throws IOException {
|
||||
// given
|
||||
createSomeMarkerFiles();
|
||||
|
||||
// then
|
||||
assertIterableEquals(CollectionUtils.createImmutableList("2020/06/01/file1.marker.MERGE",
|
||||
"2020/06/02/file2.marker.APPEND", "2020/06/03/file3.marker.CREATE"),
|
||||
markerFiles.allMarkerFilePaths().stream().sorted().collect(Collectors.toList())
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStripMarkerSuffix() {
|
||||
// Given
|
||||
final String pathPrefix = "file://" + metaClient.getMetaPath() + "/file";
|
||||
final String markerFilePath = pathPrefix + ".marker.APPEND";
|
||||
|
||||
// when-then
|
||||
assertEquals(pathPrefix, MarkerFiles.stripMarkerSuffix(markerFilePath));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,80 @@
|
||||
/*
|
||||
* 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.table.action.bootstrap;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieFileStatus;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestBootstrapUtils extends HoodieClientTestBase {
|
||||
|
||||
@Test
|
||||
public void testAllLeafFoldersWithFiles() throws IOException {
|
||||
// All directories including marker dirs.
|
||||
List<String> folders = Arrays.asList("2016/04/15", "2016/05/16", "2016/05/17");
|
||||
folders.forEach(f -> {
|
||||
try {
|
||||
metaClient.getFs().mkdirs(new Path(new Path(basePath), f));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
// Files inside partitions and marker directories
|
||||
List<String> files = Arrays.asList(
|
||||
"2016/04/15/1_1-0-1_20190528120000.parquet",
|
||||
"2016/04/15/2_1-0-1_20190528120000.parquet",
|
||||
"2016/05/16/3_1-0-1_20190528120000.parquet",
|
||||
"2016/05/16/4_1-0-1_20190528120000.parquet",
|
||||
"2016/04/17/5_1-0-1_20190528120000.parquet",
|
||||
"2016/04/17/6_1-0-1_20190528120000.parquet");
|
||||
|
||||
files.forEach(f -> {
|
||||
try {
|
||||
metaClient.getFs().create(new Path(new Path(basePath), f));
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
});
|
||||
|
||||
List<Pair<String, List<HoodieFileStatus>>> collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient,
|
||||
metaClient.getFs(), basePath, context);
|
||||
assertEquals(3, collected.size());
|
||||
collected.stream().forEach(k -> {
|
||||
assertEquals(2, k.getRight().size());
|
||||
});
|
||||
|
||||
// Simulate reading from un-partitioned dataset
|
||||
collected = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath + "/" + folders.get(0), context);
|
||||
assertEquals(1, collected.size());
|
||||
collected.stream().forEach(k -> {
|
||||
assertEquals(2, k.getRight().size());
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,459 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
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.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.testutils.Transformations;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat;
|
||||
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.MetadataMergeWriteStatus;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.parquet.avro.AvroReadSupport;
|
||||
import org.apache.parquet.hadoop.ParquetReader;
|
||||
import org.apache.spark.TaskContext;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateExpectedPartitionNumRecords;
|
||||
import static org.apache.hudi.execution.bulkinsert.TestBulkInsertInternalPartitioner.generateTestRecordsForBulkInsert;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestCopyOnWriteActionExecutor.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/exampleSchema.txt");
|
||||
|
||||
@Test
|
||||
public void testMakeNewPath() {
|
||||
String fileName = UUID.randomUUID().toString();
|
||||
String partitionPath = "2016/05/04";
|
||||
|
||||
String instantTime = makeNewCommitTime();
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
Pair<Path, String> newPathWithWriteToken = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
HoodieRecord record = mock(HoodieRecord.class);
|
||||
when(record.getPartitionPath()).thenReturn(partitionPath);
|
||||
String writeToken = FSUtils.makeWriteToken(TaskContext.getPartitionId(), TaskContext.get().stageId(),
|
||||
TaskContext.get().taskAttemptId());
|
||||
HoodieCreateHandle io = new HoodieCreateHandle(config, instantTime, table, partitionPath, fileName, supplier);
|
||||
return Pair.of(io.makeNewPath(record.getPartitionPath()), writeToken);
|
||||
}).collect().get(0);
|
||||
|
||||
assertEquals(newPathWithWriteToken.getKey().toString(), Paths.get(this.basePath, partitionPath,
|
||||
FSUtils.makeDataFileName(instantTime, newPathWithWriteToken.getRight(), fileName)).toString());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig makeHoodieClientConfig() {
|
||||
return makeHoodieClientConfigBuilder().build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||
// Prepare the AvroParquetIO
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
|
||||
}
|
||||
|
||||
// TODO (weiy): Add testcases for crossing file writing.
|
||||
@Test
|
||||
public void testUpdateRecords() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String firstCommitTime = makeNewCommitTime();
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(firstCommitTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
String partitionPath = "2016/01/31";
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Get some records belong to the same partition (2016/01/31)
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
String recordStr4 = "{\"_row_key\":\"8eb5b87d-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":51}";
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
final HoodieSparkCopyOnWriteTable cowTable = table;
|
||||
writeClient.insert(jsc.parallelize(records, 1), firstCommitTime);
|
||||
|
||||
FileStatus[] allFiles = getIncrementalFiles(partitionPath, "0", -1);
|
||||
assertEquals(1, allFiles.length);
|
||||
|
||||
// Read out the bloom filter and make sure filter can answer record exist or not
|
||||
Path parquetFilePath = allFiles[0].getPath();
|
||||
BloomFilter filter = ParquetUtils.readBloomFilterFromParquetMetadata(hadoopConf, parquetFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
assertTrue(filter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
|
||||
// Read the parquet file, check the record content
|
||||
List<GenericRecord> fileRecords = ParquetUtils.readAvroRecords(hadoopConf, parquetFilePath);
|
||||
GenericRecord newRecord;
|
||||
int index = 0;
|
||||
for (GenericRecord record : fileRecords) {
|
||||
//System.out.println("Got :" + record.get("_row_key").toString() + ", Exp :" + records.get(index).getRecordKey());
|
||||
assertEquals(records.get(index).getRecordKey(), record.get("_row_key").toString());
|
||||
index++;
|
||||
}
|
||||
|
||||
// We update the 1st record & add a new record
|
||||
String updateRecordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
RawTripTestPayload updateRowChanges1 = new RawTripTestPayload(updateRecordStr1);
|
||||
HoodieRecord updatedRecord1 = new HoodieRecord(
|
||||
new HoodieKey(updateRowChanges1.getRowKey(), updateRowChanges1.getPartitionPath()), updateRowChanges1);
|
||||
|
||||
RawTripTestPayload rowChange4 = new RawTripTestPayload(recordStr4);
|
||||
HoodieRecord insertedRecord1 =
|
||||
new HoodieRecord(new HoodieKey(rowChange4.getRowKey(), rowChange4.getPartitionPath()), rowChange4);
|
||||
|
||||
List<HoodieRecord> updatedRecords = Arrays.asList(updatedRecord1, insertedRecord1);
|
||||
|
||||
Thread.sleep(1000);
|
||||
String newCommitTime = makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
List<WriteStatus> statuses = writeClient.upsert(jsc.parallelize(updatedRecords), newCommitTime).collect();
|
||||
|
||||
allFiles = getIncrementalFiles(partitionPath, firstCommitTime, -1);
|
||||
assertEquals(1, allFiles.length);
|
||||
// verify new incremental file group is same as the previous one
|
||||
assertEquals(FSUtils.getFileId(parquetFilePath.getName()), FSUtils.getFileId(allFiles[0].getPath().getName()));
|
||||
|
||||
// Check whether the record has been updated
|
||||
Path updatedParquetFilePath = allFiles[0].getPath();
|
||||
BloomFilter updatedFilter =
|
||||
ParquetUtils.readBloomFilterFromParquetMetadata(hadoopConf, updatedParquetFilePath);
|
||||
for (HoodieRecord record : records) {
|
||||
// No change to the _row_key
|
||||
assertTrue(updatedFilter.mightContain(record.getRecordKey()));
|
||||
}
|
||||
|
||||
assertTrue(updatedFilter.mightContain(insertedRecord1.getRecordKey()));
|
||||
records.add(insertedRecord1);// add this so it can further check below
|
||||
|
||||
ParquetReader updatedReader = ParquetReader.builder(new AvroReadSupport<>(), updatedParquetFilePath).build();
|
||||
index = 0;
|
||||
while ((newRecord = (GenericRecord) updatedReader.read()) != null) {
|
||||
assertEquals(newRecord.get("_row_key").toString(), records.get(index).getRecordKey());
|
||||
if (index == 0) {
|
||||
assertEquals("15", newRecord.get("number").toString());
|
||||
}
|
||||
index++;
|
||||
}
|
||||
updatedReader.close();
|
||||
// Also check the numRecordsWritten
|
||||
WriteStatus writeStatus = statuses.get(0);
|
||||
assertEquals(1, statuses.size(), "Should be only one file generated");
|
||||
assertEquals(4, writeStatus.getStat().getNumWrites());// 3 rewritten records + 1 new record
|
||||
}
|
||||
|
||||
private FileStatus[] getIncrementalFiles(String partitionPath, String startCommitTime, int numCommitsToPull)
|
||||
throws Exception {
|
||||
// initialize parquet input format
|
||||
HoodieParquetInputFormat hoodieInputFormat = new HoodieParquetInputFormat();
|
||||
JobConf jobConf = new JobConf(hadoopConf);
|
||||
hoodieInputFormat.setConf(jobConf);
|
||||
HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
setupIncremental(jobConf, startCommitTime, numCommitsToPull);
|
||||
FileInputFormat.setInputPaths(jobConf, Paths.get(basePath, partitionPath).toString());
|
||||
return hoodieInputFormat.listStatus(jobConf);
|
||||
}
|
||||
|
||||
private void setupIncremental(JobConf jobConf, String startCommit, int numberOfCommitsToPull) {
|
||||
String modePropertyName =
|
||||
String.format(HoodieHiveUtils.HOODIE_CONSUME_MODE_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.set(modePropertyName, HoodieHiveUtils.INCREMENTAL_SCAN_MODE);
|
||||
|
||||
String startCommitTimestampName =
|
||||
String.format(HoodieHiveUtils.HOODIE_START_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.set(startCommitTimestampName, startCommit);
|
||||
|
||||
String maxCommitPulls =
|
||||
String.format(HoodieHiveUtils.HOODIE_MAX_COMMIT_PATTERN, HoodieTestUtils.RAW_TRIPS_TEST_NAME);
|
||||
jobConf.setInt(maxCommitPulls, numberOfCommitsToPull);
|
||||
}
|
||||
|
||||
private List<HoodieRecord> newHoodieRecords(int n, String time) throws Exception {
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
for (int i = 0; i < n; i++) {
|
||||
String recordStr =
|
||||
String.format("{\"_row_key\":\"%s\",\"time\":\"%s\",\"number\":%d}", UUID.randomUUID().toString(), time, i);
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
// Check if record level metadata is aggregated properly at the end of write.
|
||||
@Test
|
||||
public void testMetadataAggregateFromWriteStatus() throws Exception {
|
||||
// Prepare the AvroParquetIO
|
||||
HoodieWriteConfig config =
|
||||
makeHoodieClientConfigBuilder().withWriteStatusClass(MetadataMergeWriteStatus.class).build();
|
||||
String firstCommitTime = makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Get some records belong to the same partition (2016/01/31)
|
||||
String recordStr1 = "{\"_row_key\":\"8eb5b87a-1feh-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":12}";
|
||||
String recordStr2 = "{\"_row_key\":\"8eb5b87b-1feu-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:20:41.415Z\",\"number\":100}";
|
||||
String recordStr3 = "{\"_row_key\":\"8eb5b87c-1fej-4edd-87b4-6ec96dc405a0\","
|
||||
+ "\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":15}";
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
RawTripTestPayload rowChange1 = new RawTripTestPayload(recordStr1);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange1.getRowKey(), rowChange1.getPartitionPath()), rowChange1));
|
||||
RawTripTestPayload rowChange2 = new RawTripTestPayload(recordStr2);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange2.getRowKey(), rowChange2.getPartitionPath()), rowChange2));
|
||||
RawTripTestPayload rowChange3 = new RawTripTestPayload(recordStr3);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange3.getRowKey(), rowChange3.getPartitionPath()), rowChange3));
|
||||
|
||||
// Insert new records
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
|
||||
firstCommitTime, jsc.parallelize(records));
|
||||
List<WriteStatus> writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
Map<String, String> allWriteStatusMergedMetadataMap =
|
||||
MetadataMergeWriteStatus.mergeMetadataForWriteStatuses(writeStatuses);
|
||||
assertTrue(allWriteStatusMergedMetadataMap.containsKey("InputRecordCount_1506582000"));
|
||||
// For metadata key InputRecordCount_1506582000, value is 2 for each record. So sum of this
|
||||
// should be 2 * 3
|
||||
assertEquals("6", allWriteStatusMergedMetadataMap.get("InputRecordCount_1506582000"));
|
||||
}
|
||||
|
||||
private void verifyStatusResult(List<WriteStatus> statuses, Map<String, Long> expectedPartitionNumRecords) {
|
||||
Map<String, Long> actualPartitionNumRecords = new HashMap<>();
|
||||
|
||||
for (int i = 0; i < statuses.size(); i++) {
|
||||
WriteStatus writeStatus = statuses.get(i);
|
||||
String partitionPath = writeStatus.getPartitionPath();
|
||||
actualPartitionNumRecords.put(
|
||||
partitionPath,
|
||||
actualPartitionNumRecords.getOrDefault(partitionPath, 0L) + writeStatus.getTotalRecords());
|
||||
assertEquals(0, writeStatus.getFailedRecords().size());
|
||||
}
|
||||
|
||||
assertEquals(expectedPartitionNumRecords, actualPartitionNumRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfig();
|
||||
String instantTime = makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Case 1:
|
||||
// 10 records for partition 1, 1 record for partition 2.
|
||||
List<HoodieRecord> records = newHoodieRecords(10, "2016-01-31T03:16:41.415Z");
|
||||
records.addAll(newHoodieRecords(1, "2016-02-01T03:16:41.415Z"));
|
||||
|
||||
// Insert new records
|
||||
final List<HoodieRecord> recs2 = records;
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(recs2));
|
||||
List<WriteStatus> returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
// TODO: check the actual files and make sure 11 records, total were written.
|
||||
assertEquals(2, returnedStatuses.size());
|
||||
Map<String, Long> expectedPartitionNumRecords = new HashMap<>();
|
||||
expectedPartitionNumRecords.put("2016/01/31", 10L);
|
||||
expectedPartitionNumRecords.put("2016/02/01", 1L);
|
||||
verifyStatusResult(returnedStatuses, expectedPartitionNumRecords);
|
||||
|
||||
// Case 2:
|
||||
// 1 record for partition 1, 5 record for partition 2, 1 records for partition 3.
|
||||
records = newHoodieRecords(1, "2016-01-31T03:16:41.415Z");
|
||||
records.addAll(newHoodieRecords(5, "2016-02-01T03:16:41.415Z"));
|
||||
records.addAll(newHoodieRecords(1, "2016-02-02T03:16:41.415Z"));
|
||||
|
||||
// Insert new records
|
||||
final List<HoodieRecord> recs3 = records;
|
||||
BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(recs3));
|
||||
returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator());
|
||||
}).flatMap(Transformations::flattenAsIterator).collect();
|
||||
|
||||
assertEquals(3, returnedStatuses.size());
|
||||
expectedPartitionNumRecords.clear();
|
||||
expectedPartitionNumRecords.put("2016/01/31", 1L);
|
||||
expectedPartitionNumRecords.put("2016/02/01", 5L);
|
||||
expectedPartitionNumRecords.put("2016/02/02", 1L);
|
||||
verifyStatusResult(returnedStatuses, expectedPartitionNumRecords);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFileSizeUpsertRecords() throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.parquetMaxFileSize(64 * 1024).hfileMaxFileSize(64 * 1024)
|
||||
.parquetBlockSize(64 * 1024).parquetPageSize(64 * 1024).build()).build();
|
||||
String instantTime = makeNewCommitTime();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
// Approx 1150 records are written for block size of 64KB
|
||||
for (int i = 0; i < 2000; i++) {
|
||||
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
|
||||
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
records.add(new HoodieRecord(new HoodieKey(rowChange.getRowKey(), rowChange.getPartitionPath()), rowChange));
|
||||
}
|
||||
|
||||
// Insert new records
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(records));
|
||||
jsc.parallelize(Arrays.asList(1))
|
||||
.map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()))
|
||||
.map(Transformations::flatten).collect();
|
||||
|
||||
// Check the updated file
|
||||
int counts = 0;
|
||||
for (File file : Paths.get(basePath, "2016/01/31").toFile().listFiles()) {
|
||||
if (file.getName().endsWith(".parquet") && FSUtils.getCommitTime(file.getName()).equals(instantTime)) {
|
||||
LOG.info(file.getName() + "-" + file.length());
|
||||
counts++;
|
||||
}
|
||||
}
|
||||
assertEquals(3, counts, "If the number of records are more than 1150, then there should be a new file");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInsertUpsertWithHoodieAvroPayload() throws Exception {
|
||||
Schema schema = getSchemaFromResource(TestCopyOnWriteActionExecutor.class, "/testDataGeneratorSchema.txt");
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schema.toString())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.parquetMaxFileSize(1000 * 1024).hfileMaxFileSize(1000 * 1024).build()).build();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
final HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
String instantTime = "000";
|
||||
// Perform inserts of 100 records to test CreateHandle and BufferedExecutor
|
||||
final List<HoodieRecord> inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100);
|
||||
BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(inserts));
|
||||
final List<List<WriteStatus>> ws = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator());
|
||||
}).map(Transformations::flatten).collect();
|
||||
|
||||
WriteStatus writeStatus = ws.get(0).get(0);
|
||||
String fileId = writeStatus.getFileId();
|
||||
metaClient.getFs().create(new Path(Paths.get(basePath, ".hoodie", "000.commit").toString())).close();
|
||||
final List<HoodieRecord> updates = dataGen.generateUpdatesWithHoodieAvroPayload(instantTime, inserts);
|
||||
|
||||
String partitionPath = writeStatus.getPartitionPath();
|
||||
long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count();
|
||||
BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table,
|
||||
instantTime, jsc.parallelize(updates));
|
||||
final List<List<WriteStatus>> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> {
|
||||
return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator());
|
||||
}).map(Transformations::flatten).collect();
|
||||
assertEquals(updates.size() - numRecordsInPartition, updateStatus.get(0).get(0).getTotalErrorRecords());
|
||||
}
|
||||
|
||||
public void testBulkInsertRecords(String bulkInsertMode) throws Exception {
|
||||
HoodieWriteConfig config = HoodieWriteConfig.newBuilder()
|
||||
.withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withBulkInsertParallelism(2).withBulkInsertSortMode(bulkInsertMode).build();
|
||||
String instantTime = makeNewCommitTime();
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
writeClient.startCommitWithTime(instantTime);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
// Insert new records
|
||||
final JavaRDD<HoodieRecord> inputRecords = generateTestRecordsForBulkInsert(jsc);
|
||||
SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor(
|
||||
context, config, table, instantTime, inputRecords, Option.empty());
|
||||
List<WriteStatus> returnedStatuses = ((JavaRDD<WriteStatus>)bulkInsertExecutor.execute().getWriteStatuses()).collect();
|
||||
verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords));
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = "[{index}] {0}")
|
||||
@ValueSource(strings = {"global_sort", "partition_sort", "none"})
|
||||
public void testBulkInsertRecordsWithGlobalSort(String bulkInsertMode) throws Exception {
|
||||
testBulkInsertRecords(bulkInsertMode);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,186 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.spark.Partition;
|
||||
import org.apache.spark.api.java.JavaPairRDD;
|
||||
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.junit.jupiter.api.extension.ExtendWith;
|
||||
import org.mockito.Mock;
|
||||
import org.mockito.junit.jupiter.MockitoExtension;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.ArgumentMatchers.anyInt;
|
||||
import static org.mockito.ArgumentMatchers.anyString;
|
||||
import static org.mockito.ArgumentMatchers.eq;
|
||||
import static org.mockito.Mockito.doNothing;
|
||||
import static org.mockito.Mockito.doReturn;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.never;
|
||||
import static org.mockito.Mockito.times;
|
||||
import static org.mockito.Mockito.verify;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
@ExtendWith(MockitoExtension.class)
|
||||
public class TestDeleteHelper {
|
||||
|
||||
private enum CombineTestMode {
|
||||
None, GlobalIndex, NoneGlobalIndex;
|
||||
}
|
||||
|
||||
private static final String BASE_PATH = "/tmp/";
|
||||
private static final boolean WITH_COMBINE = true;
|
||||
private static final boolean WITHOUT_COMBINE = false;
|
||||
private static final int DELETE_PARALLELISM = 200;
|
||||
|
||||
@Mock private SparkHoodieBloomIndex index;
|
||||
@Mock private HoodieTable<EmptyHoodieRecordPayload,JavaRDD<HoodieRecord>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table;
|
||||
@Mock private BaseSparkCommitActionExecutor<EmptyHoodieRecordPayload> executor;
|
||||
@Mock private HoodieWriteMetadata metadata;
|
||||
@Mock private JavaPairRDD keyPairs;
|
||||
@Mock private JavaSparkContext jsc;
|
||||
@Mock private HoodieSparkEngineContext context;
|
||||
|
||||
private JavaRDD<HoodieKey> rddToDelete;
|
||||
private HoodieWriteConfig config;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() {
|
||||
when(table.getIndex()).thenReturn(index);
|
||||
when(context.getJavaSparkContext()).thenReturn(jsc);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithEmptyRDDShouldNotExecute() {
|
||||
rddToDelete = mockEmptyHoodieKeyRdd();
|
||||
config = newWriteConfig(WITHOUT_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(rddToDelete, never()).repartition(DELETE_PARALLELISM);
|
||||
verifyNoDeleteExecution();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() {
|
||||
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None);
|
||||
config = newWriteConfig(WITHOUT_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM);
|
||||
verifyDeleteExecution();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() {
|
||||
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex);
|
||||
config = newWriteConfig(WITH_COMBINE);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM);
|
||||
verifyDeleteExecution();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void deleteWithCombineShouldRepartitionForNonEmptyRddAndGlobalIndex() {
|
||||
rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.GlobalIndex);
|
||||
config = newWriteConfig(WITH_COMBINE);
|
||||
when(index.isGlobal()).thenReturn(true);
|
||||
|
||||
SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor);
|
||||
|
||||
verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM));
|
||||
verifyDeleteExecution();
|
||||
}
|
||||
|
||||
private void verifyDeleteExecution() {
|
||||
verify(executor, times(1)).execute(any());
|
||||
verify(metadata, times(1)).setIndexLookupDuration(any());
|
||||
}
|
||||
|
||||
private void verifyNoDeleteExecution() {
|
||||
verify(executor, never()).execute(any());
|
||||
}
|
||||
|
||||
private HoodieWriteConfig newWriteConfig(boolean combine) {
|
||||
return HoodieWriteConfig.newBuilder()
|
||||
.combineDeleteInput(combine)
|
||||
.withPath(BASE_PATH)
|
||||
.withDeleteParallelism(DELETE_PARALLELISM)
|
||||
.build();
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieKey> newHoodieKeysRddMock(int howMany, CombineTestMode combineMode) {
|
||||
JavaRDD<HoodieKey> keysToDelete = mock(JavaRDD.class);
|
||||
|
||||
JavaRDD recordsRdd = mock(JavaRDD.class);
|
||||
when(recordsRdd.filter(any())).thenReturn(recordsRdd);
|
||||
when(recordsRdd.isEmpty()).thenReturn(howMany <= 0);
|
||||
when(index.tagLocation(any(), any(), any())).thenReturn(recordsRdd);
|
||||
|
||||
if (combineMode == CombineTestMode.GlobalIndex) {
|
||||
when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs);
|
||||
when(keyPairs.values()).thenReturn(keysToDelete);
|
||||
when(keysToDelete.keyBy(any())).thenReturn(keyPairs);
|
||||
} else if (combineMode == CombineTestMode.NoneGlobalIndex) {
|
||||
when(keysToDelete.distinct(anyInt())).thenReturn(keysToDelete);
|
||||
} else if (combineMode == CombineTestMode.None) {
|
||||
List<Partition> parts = mock(List.class);
|
||||
when(parts.isEmpty()).thenReturn(howMany <= 0);
|
||||
when(keysToDelete.repartition(anyInt())).thenReturn(keysToDelete);
|
||||
when(keysToDelete.partitions()).thenReturn(parts);
|
||||
}
|
||||
|
||||
when(keysToDelete.map(any())).thenReturn(recordsRdd);
|
||||
when(executor.execute(any())).thenReturn(metadata);
|
||||
return keysToDelete;
|
||||
}
|
||||
|
||||
private JavaRDD<HoodieKey> mockEmptyHoodieKeyRdd() {
|
||||
JavaRDD<HoodieKey> emptyRdd = mock(JavaRDD.class);
|
||||
doReturn(true).when(emptyRdd).isEmpty();
|
||||
doReturn(Collections.emptyList()).when(emptyRdd).partitions();
|
||||
doReturn(emptyRdd).when(emptyRdd).map(any());
|
||||
|
||||
doReturn(emptyRdd).when(index).tagLocation(any(), any(), any());
|
||||
doReturn(emptyRdd).when(emptyRdd).filter(any());
|
||||
|
||||
doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString());
|
||||
doReturn(emptyRdd).when(jsc).emptyRDD();
|
||||
return emptyRdd;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,298 @@
|
||||
/*
|
||||
* 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.table.action.commit;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.WorkloadProfile;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import scala.Tuple2;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.generateFakeHoodieWriteStat;
|
||||
import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource;
|
||||
import static org.apache.hudi.table.action.commit.UpsertPartitioner.averageBytesPerRecord;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.mockito.ArgumentMatchers.any;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
public class TestUpsertPartitioner extends HoodieClientTestBase {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(TestUpsertPartitioner.class);
|
||||
private static final Schema SCHEMA = getSchemaFromResource(TestUpsertPartitioner.class, "/exampleSchema.txt");
|
||||
|
||||
private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts, int numUpdates, int fileSize,
|
||||
String testPartitionPath, boolean autoSplitInserts) throws Exception {
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(smallFileSize)
|
||||
.insertSplitSize(100).autoTuneInsertSplits(autoSplitInserts).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1000 * 1024).parquetMaxFileSize(1000 * 1024).build())
|
||||
.build();
|
||||
|
||||
FileCreateUtils.createCommit(basePath, "001");
|
||||
FileCreateUtils.createBaseFile(basePath, testPartitionPath, "001", "file1", fileSize);
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", numInserts);
|
||||
List<HoodieRecord> updateRecords = dataGenerator.generateUpdates("001", numUpdates);
|
||||
for (HoodieRecord updateRec : updateRecords) {
|
||||
updateRec.unseal();
|
||||
updateRec.setCurrentLocation(new HoodieRecordLocation("001", "file1"));
|
||||
updateRec.seal();
|
||||
}
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
records.addAll(insertRecords);
|
||||
records.addAll(updateRecords);
|
||||
WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(records)));
|
||||
UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config);
|
||||
assertEquals(0, partitioner.getPartition(
|
||||
new Tuple2<>(updateRecords.get(0).getKey(), Option.ofNullable(updateRecords.get(0).getCurrentLocation()))),
|
||||
"Update record should have gone to the 1 update partition");
|
||||
return partitioner;
|
||||
}
|
||||
|
||||
private static List<HoodieInstant> setupHoodieInstants() {
|
||||
List<HoodieInstant> instants = new ArrayList<>();
|
||||
instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts1"));
|
||||
instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts2"));
|
||||
instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts3"));
|
||||
instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts4"));
|
||||
instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts5"));
|
||||
Collections.reverse(instants);
|
||||
return instants;
|
||||
}
|
||||
|
||||
private static List<HoodieWriteStat> generateCommitStatWith(int totalRecordsWritten, int totalBytesWritten) {
|
||||
List<HoodieWriteStat> writeStatsList = generateFakeHoodieWriteStat(5);
|
||||
// clear all record and byte stats except for last entry.
|
||||
for (int i = 0; i < writeStatsList.size() - 1; i++) {
|
||||
HoodieWriteStat writeStat = writeStatsList.get(i);
|
||||
writeStat.setNumWrites(0);
|
||||
writeStat.setTotalWriteBytes(0);
|
||||
}
|
||||
HoodieWriteStat lastWriteStat = writeStatsList.get(writeStatsList.size() - 1);
|
||||
lastWriteStat.setTotalWriteBytes(totalBytesWritten);
|
||||
lastWriteStat.setNumWrites(totalRecordsWritten);
|
||||
return writeStatsList;
|
||||
}
|
||||
|
||||
private static HoodieCommitMetadata generateCommitMetadataWith(int totalRecordsWritten, int totalBytesWritten) {
|
||||
List<HoodieWriteStat> fakeHoodieWriteStats = generateCommitStatWith(totalRecordsWritten, totalBytesWritten);
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
fakeHoodieWriteStats.forEach(stat -> commitMetadata.addWriteStat(stat.getPartitionPath(), stat));
|
||||
return commitMetadata;
|
||||
}
|
||||
|
||||
/*
|
||||
* This needs to be a stack so we test all cases when either/both recordsWritten ,bytesWritten is zero before a non
|
||||
* zero averageRecordSize can be computed.
|
||||
*/
|
||||
private static LinkedList<Option<byte[]>> generateCommitMetadataList() throws IOException {
|
||||
LinkedList<Option<byte[]>> commits = new LinkedList<>();
|
||||
// First commit with non zero records and bytes
|
||||
commits.push(Option.of(generateCommitMetadataWith(2000, 10000).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Second commit with non zero records and bytes
|
||||
commits.push(Option.of(generateCommitMetadataWith(1500, 7500).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Third commit with a small file
|
||||
commits.push(Option.of(generateCommitMetadataWith(100, 500).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Fourth commit with both zero records and zero bytes
|
||||
commits.push(Option.of(generateCommitMetadataWith(0, 0).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Fifth commit with zero records
|
||||
commits.push(Option.of(generateCommitMetadataWith(0, 1500).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
// Sixth commit with zero bytes
|
||||
commits.push(Option.of(generateCommitMetadataWith(2500, 0).toJsonString().getBytes(StandardCharsets.UTF_8)));
|
||||
return commits;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAverageBytesPerRecordForNonEmptyCommitTimeLine() throws Exception {
|
||||
HoodieTimeline commitTimeLine = mock(HoodieTimeline.class);
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1000).build())
|
||||
.build();
|
||||
when(commitTimeLine.empty()).thenReturn(false);
|
||||
when(commitTimeLine.getReverseOrderedInstants()).thenReturn(setupHoodieInstants().stream());
|
||||
LinkedList<Option<byte[]>> commits = generateCommitMetadataList();
|
||||
when(commitTimeLine.getInstantDetails(any(HoodieInstant.class))).thenAnswer(invocationOnMock -> commits.pop());
|
||||
long expectAvgSize = (long) Math.ceil((1.0 * 7500) / 1500);
|
||||
long actualAvgSize = averageBytesPerRecord(commitTimeLine, config);
|
||||
assertEquals(expectAvgSize, actualAvgSize);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAverageBytesPerRecordForEmptyCommitTimeLine() throws Exception {
|
||||
HoodieTimeline commitTimeLine = mock(HoodieTimeline.class);
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder().build();
|
||||
when(commitTimeLine.empty()).thenReturn(true);
|
||||
long expectAvgSize = config.getCopyOnWriteRecordSizeEstimate();
|
||||
long actualAvgSize = averageBytesPerRecord(commitTimeLine, config);
|
||||
assertEquals(expectAvgSize, actualAvgSize);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitioner() throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
// Inserts + Updates... Check all updates go together & inserts subsplit
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(0, 200, 100, 1024, testPartitionPath, false);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
assertEquals(2, insertBuckets.size(), "Total of 2 insert buckets");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testPartitionWeight() throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
int totalInsertNum = 2000;
|
||||
|
||||
HoodieWriteConfig config = makeHoodieClientConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(0)
|
||||
.insertSplitSize(totalInsertNum / 2).autoTuneInsertSplits(false).build()).build();
|
||||
|
||||
FileCreateUtils.createCommit(basePath, "001");
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, metaClient);
|
||||
HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {testPartitionPath});
|
||||
List<HoodieRecord> insertRecords = dataGenerator.generateInserts("001", totalInsertNum);
|
||||
|
||||
WorkloadProfile profile = new WorkloadProfile(buildProfile(jsc.parallelize(insertRecords)));
|
||||
UpsertPartitioner partitioner = new UpsertPartitioner(profile, context, table, config);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
float bucket0Weight = 0.2f;
|
||||
InsertBucketCumulativeWeightPair pair = insertBuckets.remove(0);
|
||||
pair.getKey().weight = bucket0Weight;
|
||||
pair.setValue(new Double(bucket0Weight));
|
||||
insertBuckets.add(0, pair);
|
||||
|
||||
InsertBucketCumulativeWeightPair pair1 = insertBuckets.remove(1);
|
||||
pair1.getKey().weight = 1 - bucket0Weight;
|
||||
pair1.setValue(new Double(1));
|
||||
insertBuckets.add(1, pair1);
|
||||
|
||||
Map<Integer, Integer> partition2numRecords = new HashMap<Integer, Integer>();
|
||||
for (HoodieRecord hoodieRecord: insertRecords) {
|
||||
int partition = partitioner.getPartition(new Tuple2<>(
|
||||
hoodieRecord.getKey(), Option.ofNullable(hoodieRecord.getCurrentLocation())));
|
||||
if (!partition2numRecords.containsKey(partition)) {
|
||||
partition2numRecords.put(partition, 0);
|
||||
}
|
||||
partition2numRecords.put(partition, partition2numRecords.get(partition) + 1);
|
||||
}
|
||||
|
||||
assertTrue(partition2numRecords.get(0) < partition2numRecords.get(1),
|
||||
"The insert num of bucket1 should more than bucket0");
|
||||
assertTrue(partition2numRecords.get(0) + partition2numRecords.get(1) == totalInsertNum,
|
||||
"The total insert records should be " + totalInsertNum);
|
||||
assertEquals(String.valueOf(bucket0Weight),
|
||||
String.format("%.1f", (partition2numRecords.get(0) * 1.0f / totalInsertNum)),
|
||||
"The weight of bucket0 should be " + bucket0Weight);
|
||||
assertEquals(String.valueOf(1 - bucket0Weight),
|
||||
String.format("%.1f", (partition2numRecords.get(1) * 1.0f / totalInsertNum)),
|
||||
"The weight of bucket1 should be " + (1 - bucket0Weight));
|
||||
}
|
||||
|
||||
private void assertInsertBuckets(Double[] weights,
|
||||
Double[] cumulativeWeights,
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets) {
|
||||
for (int i = 0; i < weights.length; i++) {
|
||||
assertEquals(i, insertBuckets.get(i).getKey().bucketNumber,
|
||||
String.format("BucketNumber of insert bucket %d must be same as %d", i, i));
|
||||
assertEquals(weights[i], insertBuckets.get(i).getKey().weight, 0.01,
|
||||
String.format("Insert bucket %d should have weight %.1f", i, weights[i]));
|
||||
assertEquals(cumulativeWeights[i], insertBuckets.get(i).getValue(), 0.01,
|
||||
String.format("Insert bucket %d should have cumulativeWeight %.1f", i, cumulativeWeights[i]));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpsertPartitionerWithSmallInsertHandling() throws Exception {
|
||||
final String testPartitionPath = "2016/09/26";
|
||||
// Inserts + Updates .. Check updates go together & inserts subsplit, after expanding
|
||||
// smallest file
|
||||
UpsertPartitioner partitioner = getUpsertPartitioner(1000 * 1024, 400, 100, 800 * 1024, testPartitionPath, false);
|
||||
List<InsertBucketCumulativeWeightPair> insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
assertEquals(3, partitioner.numPartitions(), "Should have 3 partitions");
|
||||
assertEquals(BucketType.UPDATE, partitioner.getBucketInfo(0).bucketType,
|
||||
"Bucket 0 is UPDATE");
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(1).bucketType,
|
||||
"Bucket 1 is INSERT");
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(2).bucketType,
|
||||
"Bucket 2 is INSERT");
|
||||
assertEquals(3, insertBuckets.size(), "Total of 3 insert buckets");
|
||||
|
||||
Double[] weights = { 0.5, 0.25, 0.25};
|
||||
Double[] cumulativeWeights = { 0.5, 0.75, 1.0};
|
||||
assertInsertBuckets(weights, cumulativeWeights, insertBuckets);
|
||||
|
||||
// Now with insert split size auto tuned
|
||||
partitioner = getUpsertPartitioner(1000 * 1024, 2400, 100, 800 * 1024, testPartitionPath, true);
|
||||
insertBuckets = partitioner.getInsertBuckets(testPartitionPath);
|
||||
|
||||
assertEquals(4, partitioner.numPartitions(), "Should have 4 partitions");
|
||||
assertEquals(BucketType.UPDATE, partitioner.getBucketInfo(0).bucketType,
|
||||
"Bucket 0 is UPDATE");
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(1).bucketType,
|
||||
"Bucket 1 is INSERT");
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(2).bucketType,
|
||||
"Bucket 2 is INSERT");
|
||||
assertEquals(BucketType.INSERT, partitioner.getBucketInfo(3).bucketType,
|
||||
"Bucket 3 is INSERT");
|
||||
assertEquals(4, insertBuckets.size(), "Total of 4 insert buckets");
|
||||
|
||||
weights = new Double[] { 0.08, 0.31, 0.31, 0.31};
|
||||
cumulativeWeights = new Double[] { 0.08, 0.39, 0.69, 1.0};
|
||||
assertInsertBuckets(weights, cumulativeWeights, insertBuckets);
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder makeHoodieClientConfigBuilder() {
|
||||
// Prepare the AvroParquetIO
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(SCHEMA.toString());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,245 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieFileGroupId;
|
||||
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.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
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.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class CompactionTestBase extends HoodieClientTestBase {
|
||||
|
||||
protected HoodieWriteConfig.Builder getConfigBuilder(Boolean autoCommit) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath)
|
||||
.withSchema(TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withAutoCommit(autoCommit).withAssumeDatePartitioning(true)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder()
|
||||
.hfileMaxFileSize(1024 * 1024 * 1024).parquetMaxFileSize(1024 * 1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
/**
|
||||
* HELPER METHODS FOR TESTING.
|
||||
**/
|
||||
protected void validateDeltaCommit(String latestDeltaCommit, final Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation,
|
||||
HoodieWriteConfig cfg) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable table = getHoodieTable(metaClient, cfg);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table);
|
||||
fileSliceList.forEach(fileSlice -> {
|
||||
Pair<String, HoodieCompactionOperation> opPair = fgIdToCompactionOperation.get(fileSlice.getFileGroupId());
|
||||
if (opPair != null) {
|
||||
assertEquals(fileSlice.getBaseInstantTime(), opPair.getKey(), "Expect baseInstant to match compaction Instant");
|
||||
assertTrue(fileSlice.getLogFiles().count() > 0,
|
||||
"Expect atleast one log file to be present where the latest delta commit was written");
|
||||
assertFalse(fileSlice.getBaseFile().isPresent(), "Expect no data-file to be present");
|
||||
} else {
|
||||
assertTrue(fileSlice.getBaseInstantTime().compareTo(latestDeltaCommit) <= 0,
|
||||
"Expect baseInstant to be less than or equal to latestDeltaCommit");
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
protected List<HoodieRecord> runNextDeltaCommits(SparkRDDWriteClient client, final HoodieReadClient readClient, List<String> deltaInstants,
|
||||
List<HoodieRecord> records, HoodieWriteConfig cfg, boolean insertFirst, List<String> expPendingCompactionInstants)
|
||||
throws Exception {
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
List<Pair<String, HoodieCompactionPlan>> pendingCompactions = readClient.getPendingCompactions();
|
||||
List<String> gotPendingCompactionInstants =
|
||||
pendingCompactions.stream().map(pc -> pc.getKey()).sorted().collect(Collectors.toList());
|
||||
assertEquals(expPendingCompactionInstants, gotPendingCompactionInstants);
|
||||
|
||||
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> fgIdToCompactionOperation =
|
||||
CompactionUtils.getAllPendingCompactionOperations(metaClient);
|
||||
|
||||
if (insertFirst) {
|
||||
// Use first instant for inserting records
|
||||
String firstInstant = deltaInstants.get(0);
|
||||
deltaInstants = deltaInstants.subList(1, deltaInstants.size());
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
client.startCommitWithTime(firstInstant);
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, firstInstant);
|
||||
List<WriteStatus> statusList = statuses.collect();
|
||||
|
||||
if (!cfg.shouldAutoCommit()) {
|
||||
client.commit(firstInstant, statuses);
|
||||
}
|
||||
assertNoWriteErrors(statusList);
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
List<HoodieBaseFile> dataFilesToRead = getCurrentLatestDataFiles(hoodieTable, cfg);
|
||||
assertTrue(dataFilesToRead.stream().findAny().isPresent(),
|
||||
"should list the parquet files we wrote in the delta commit");
|
||||
validateDeltaCommit(firstInstant, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
|
||||
int numRecords = records.size();
|
||||
for (String instantTime : deltaInstants) {
|
||||
records = dataGen.generateUpdates(instantTime, numRecords);
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
createNextDeltaCommit(instantTime, records, client, metaClient, cfg, false);
|
||||
validateDeltaCommit(instantTime, fgIdToCompactionOperation, cfg);
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
protected void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteConfig cfg) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant);
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstants()
|
||||
.filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get();
|
||||
assertTrue(instant.isInflight(), "Instant must be marked inflight");
|
||||
}
|
||||
|
||||
protected void scheduleCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieWriteConfig cfg) {
|
||||
client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get();
|
||||
assertEquals(compactionInstantTime, instant.getTimestamp(), "Last compaction instant must be the one set");
|
||||
}
|
||||
|
||||
protected void scheduleAndExecuteCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table,
|
||||
HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
executeCompaction(compactionInstantTime, client, table, cfg, expectedNumRecs, hasDeltaCommitAfterPendingCompaction);
|
||||
}
|
||||
|
||||
protected void executeCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table,
|
||||
HoodieWriteConfig cfg, int expectedNumRecs, boolean hasDeltaCommitAfterPendingCompaction) throws IOException {
|
||||
|
||||
client.compact(compactionInstantTime);
|
||||
List<FileSlice> fileSliceList = getCurrentLatestFileSlices(table);
|
||||
assertTrue(fileSliceList.stream().findAny().isPresent(), "Ensure latest file-slices are not empty");
|
||||
assertFalse(fileSliceList.stream()
|
||||
.anyMatch(fs -> !fs.getBaseInstantTime().equals(compactionInstantTime)),
|
||||
"Verify all file-slices have base-instant same as compaction instant");
|
||||
assertFalse(fileSliceList.stream().anyMatch(fs -> !fs.getBaseFile().isPresent()),
|
||||
"Verify all file-slices have data-files");
|
||||
|
||||
if (hasDeltaCommitAfterPendingCompaction) {
|
||||
assertFalse(fileSliceList.stream().anyMatch(fs -> fs.getLogFiles().count() == 0),
|
||||
"Verify all file-slices have atleast one log-file");
|
||||
} else {
|
||||
assertFalse(fileSliceList.stream().anyMatch(fs -> fs.getLogFiles().count() > 0),
|
||||
"Verify all file-slices have no log-files");
|
||||
}
|
||||
|
||||
// verify that there is a commit
|
||||
table = getHoodieTable(new HoodieTableMetaClient(hadoopConf, cfg.getBasePath(), true), cfg);
|
||||
HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants();
|
||||
String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp();
|
||||
assertEquals(latestCompactionCommitTime, compactionInstantTime,
|
||||
"Expect compaction instant time to be the latest commit time");
|
||||
assertEquals(expectedNumRecs,
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, "000"),
|
||||
"Must contain expected records");
|
||||
|
||||
}
|
||||
|
||||
protected List<WriteStatus> createNextDeltaCommit(String instantTime, List<HoodieRecord> records, SparkRDDWriteClient client,
|
||||
HoodieTableMetaClient metaClient, HoodieWriteConfig cfg, boolean skipCommit) {
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
client.startCommitWithTime(instantTime);
|
||||
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, instantTime);
|
||||
List<WriteStatus> statusList = statuses.collect();
|
||||
assertNoWriteErrors(statusList);
|
||||
if (!cfg.shouldAutoCommit() && !skipCommit) {
|
||||
client.commit(instantTime, statuses);
|
||||
}
|
||||
|
||||
Option<HoodieInstant> deltaCommit =
|
||||
metaClient.getActiveTimeline().reload().getDeltaCommitTimeline().filterCompletedInstants().lastInstant();
|
||||
if (skipCommit && !cfg.shouldAutoCommit()) {
|
||||
assertTrue(deltaCommit.get().getTimestamp().compareTo(instantTime) < 0,
|
||||
"Delta commit should not be latest instant");
|
||||
} else {
|
||||
assertTrue(deltaCommit.isPresent());
|
||||
assertEquals(instantTime, deltaCommit.get().getTimestamp(), "Delta commit should be latest instant");
|
||||
}
|
||||
return statusList;
|
||||
}
|
||||
|
||||
protected List<HoodieBaseFile> getCurrentLatestDataFiles(HoodieTable table, HoodieWriteConfig cfg) throws IOException {
|
||||
FileStatus[] allFiles = HoodieTestUtils.listAllDataFilesInPath(table.getMetaClient().getFs(), cfg.getBasePath());
|
||||
HoodieTableFileSystemView view =
|
||||
getHoodieTableFileSystemView(table.getMetaClient(), table.getCompletedCommitsTimeline(), allFiles);
|
||||
return view.getLatestBaseFiles().collect(Collectors.toList());
|
||||
}
|
||||
|
||||
protected List<FileSlice> getCurrentLatestFileSlices(HoodieTable table) {
|
||||
HoodieTableFileSystemView view = new HoodieTableFileSystemView(table.getMetaClient(),
|
||||
table.getMetaClient().getActiveTimeline().reload().getCommitsAndCompactionTimeline());
|
||||
return Arrays.stream(HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS)
|
||||
.flatMap(view::getLatestFileSlices).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,335 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Test Cases for Async Compaction and Ingestion interaction.
|
||||
*/
|
||||
public class TestAsyncCompaction extends CompactionTestBase {
|
||||
|
||||
private HoodieWriteConfig getConfig(Boolean autoCommit) {
|
||||
return getConfigBuilder(autoCommit).build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackForInflightCompaction() throws Exception {
|
||||
// Rollback inflight compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(),
|
||||
"Pending Compaction instant has expected instant time");
|
||||
assertEquals(State.REQUESTED, pendingCompactionInstant.getState(), "Pending Compaction instant has expected state");
|
||||
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, cfg);
|
||||
|
||||
// Reload and rollback inflight compaction
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient);
|
||||
// hoodieTable.rollback(jsc,
|
||||
// new HoodieInstant(true, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), false);
|
||||
|
||||
client.rollbackInflightCompaction(
|
||||
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime), hoodieTable);
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline()
|
||||
.getInstants().findFirst().get();
|
||||
assertEquals("compaction", pendingCompactionInstant.getAction());
|
||||
assertEquals(State.REQUESTED, pendingCompactionInstant.getState());
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp());
|
||||
|
||||
// We indirectly test for the race condition where a inflight instant was first deleted then created new. Every
|
||||
// time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie
|
||||
// reads compaction plan from aux path which is untouched). TO test for regression, we simply get file status
|
||||
// and look at the file size
|
||||
FileStatus fstatus =
|
||||
metaClient.getFs().getFileStatus(new Path(metaClient.getMetaPath(), pendingCompactionInstant.getFileName()));
|
||||
assertTrue(fstatus.getLen() > 0);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRollbackInflightIngestionWithPendingCompaction() throws Exception {
|
||||
// Rollback inflight ingestion when there is pending compaction
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String inflightInstantTime = "006";
|
||||
String nextInflightInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(),
|
||||
"Pending Compaction instant has expected instant time");
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals(inflightInstantTime, inflightInstant.getTimestamp(), "inflight instant has expected instant time");
|
||||
|
||||
// This should rollback
|
||||
client.startCommitWithTime(nextInflightInstantTime);
|
||||
|
||||
// Validate
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals(inflightInstant.getTimestamp(), nextInflightInstantTime, "inflight instant has expected instant time");
|
||||
assertEquals(1, metaClient.getActiveTimeline()
|
||||
.filterPendingExcludingCompaction().getInstants().count(),
|
||||
"Expect only one inflight instant");
|
||||
// Expect pending Compaction to be present
|
||||
pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(),
|
||||
"Pending Compaction instant has expected instant time");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInflightCompaction() throws Exception {
|
||||
// There is inflight compaction. Subsequent compaction run must work correctly
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
// Schedule and mark compaction instant as inflight
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
moveCompactionFromRequestedToInflight(compactionInstantTime, cfg);
|
||||
|
||||
// Complete ingestions
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(thirdInstantTime, fourthInstantTime), records, cfg, false,
|
||||
Arrays.asList(compactionInstantTime));
|
||||
|
||||
// execute inflight compaction
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleIngestionBeforePendingCompaction() throws Exception {
|
||||
// Case: Failure case. Latest pending compaction instant time must be earlier than this instant time
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String failedInstantTime = "005";
|
||||
String compactionInstantTime = "006";
|
||||
int numRecs = 2000;
|
||||
|
||||
final List<HoodieRecord> initalRecords = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
final List<HoodieRecord> records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), initalRecords, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieInstant pendingCompactionInstant =
|
||||
metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get();
|
||||
assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), "Pending Compaction instant has expected instant time");
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(failedInstantTime), records, cfg, false,
|
||||
Arrays.asList(compactionInstantTime));
|
||||
}, "Latest pending compaction instant time must be earlier than this instant time");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleCompactionAfterPendingIngestion() throws Exception {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String inflightInstantTime = "005";
|
||||
String compactionInstantTime = "006";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
createNextDeltaCommit(inflightInstantTime, records, client, metaClient, cfg, true);
|
||||
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieInstant inflightInstant =
|
||||
metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get();
|
||||
assertEquals(inflightInstantTime, inflightInstant.getTimestamp(), "inflight instant has expected instant time");
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
}, "Earliest ingestion inflight instant time must be later than compaction time");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScheduleCompactionWithOlderOrSameTimestamp() throws Exception {
|
||||
// Case: Failure case. Earliest ingestion inflight instant time must be later than compaction time
|
||||
|
||||
HoodieWriteConfig cfg = getConfig(false);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
|
||||
final String firstInstantTime = "001";
|
||||
final String secondInstantTime = "004";
|
||||
final String compactionInstantTime = "002";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
}, "Compaction Instant to be scheduled cannot have older timestamp");
|
||||
|
||||
// Schedule with timestamp same as that of committed instant
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
// Schedule compaction but do not run them
|
||||
scheduleCompaction(secondInstantTime, client, cfg);
|
||||
}, "Compaction Instant to be scheduled cannot have same timestamp as committed instant");
|
||||
|
||||
final String compactionInstantTime2 = "006";
|
||||
scheduleCompaction(compactionInstantTime2, client, cfg);
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
// Schedule compaction with the same times as a pending compaction
|
||||
scheduleCompaction(secondInstantTime, client, cfg);
|
||||
}, "Compaction Instant to be scheduled cannot have same timestamp as a pending compaction");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionAfterTwoDeltaCommits() throws Exception {
|
||||
// No Delta Commits after compaction request
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleAndExecuteCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, false);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInterleavedCompaction() throws Exception {
|
||||
// Case: Two delta commits before and after compaction schedule
|
||||
HoodieWriteConfig cfg = getConfig(true);
|
||||
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);) {
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
String firstInstantTime = "001";
|
||||
String secondInstantTime = "004";
|
||||
String compactionInstantTime = "005";
|
||||
String thirdInstantTime = "006";
|
||||
String fourthInstantTime = "007";
|
||||
|
||||
int numRecs = 2000;
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(firstInstantTime, numRecs);
|
||||
records = runNextDeltaCommits(client, readClient, Arrays.asList(firstInstantTime, secondInstantTime), records, cfg, true,
|
||||
new ArrayList<>());
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
HoodieTable hoodieTable = getHoodieTable(metaClient, cfg);
|
||||
scheduleCompaction(compactionInstantTime, client, cfg);
|
||||
|
||||
runNextDeltaCommits(client, readClient, Arrays.asList(thirdInstantTime, fourthInstantTime), records, cfg, false,
|
||||
Arrays.asList(compactionInstantTime));
|
||||
executeCompaction(compactionInstantTime, client, hoodieTable, cfg, numRecs, true);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,197 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
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.HoodieInstant.State;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieMemoryConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.index.bloom.SparkHoodieBloomIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestHarness;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.FileCreateUtils.createDeltaCommit;
|
||||
import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit;
|
||||
import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieCompactor extends HoodieClientTestHarness {
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private HoodieTableMetaClient metaClient;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
// Initialize a local spark env
|
||||
initSparkContexts();
|
||||
|
||||
// Create a temp folder as the base path
|
||||
initPath();
|
||||
hadoopConf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
fs = FSUtils.getFs(basePath, hadoopConf);
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
||||
initTestDataGenerator();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder()
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
private HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024)
|
||||
.withInlineCompaction(false).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.withMemoryConfig(HoodieMemoryConfig.newBuilder().withMaxDFSStreamBufferSize(1 * 1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionOnCopyOnWriteFail() throws Exception {
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.COPY_ON_WRITE);
|
||||
HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
assertThrows(HoodieNotSupportedException.class, () -> {
|
||||
table.scheduleCompaction(context, compactionInstantTime, Option.empty());
|
||||
table.compact(context, compactionInstantTime);
|
||||
});
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionEmpty() throws Exception {
|
||||
HoodieWriteConfig config = getConfig();
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient);
|
||||
HoodieTable table = HoodieSparkTable.create(getConfig(), context, metaClient);
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config);) {
|
||||
|
||||
String newCommitTime = writeClient.startCommit();
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
String compactionInstantTime = HoodieActiveTimeline.createNewInstantTime();
|
||||
Option<HoodieCompactionPlan> plan = table.scheduleCompaction(context, compactionInstantTime, Option.empty());
|
||||
assertFalse(plan.isPresent(), "If there is nothing to compact, result will be empty");
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testWriteStatusContentsAfterCompaction() throws Exception {
|
||||
// insert 100 records
|
||||
HoodieWriteConfig config = getConfig();
|
||||
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
|
||||
String newCommitTime = "100";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 100);
|
||||
JavaRDD<HoodieRecord> recordsRDD = jsc.parallelize(records, 1);
|
||||
writeClient.insert(recordsRDD, newCommitTime).collect();
|
||||
|
||||
// Update all the 100 records
|
||||
HoodieTable table = HoodieSparkTable.create(config, context);
|
||||
newCommitTime = "101";
|
||||
writeClient.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> updatedRecords = dataGen.generateUpdates(newCommitTime, records);
|
||||
JavaRDD<HoodieRecord> updatedRecordsRDD = jsc.parallelize(updatedRecords, 1);
|
||||
HoodieIndex index = new SparkHoodieBloomIndex<>(config);
|
||||
updatedRecords = ((JavaRDD<HoodieRecord>)index.tagLocation(updatedRecordsRDD, context, table)).collect();
|
||||
|
||||
// Write them to corresponding avro logfiles. Also, set the state transition properly.
|
||||
HoodieTestUtils.writeRecordsToLogFiles(fs, metaClient.getBasePath(),
|
||||
HoodieTestDataGenerator.AVRO_SCHEMA_WITH_METADATA_FIELDS, updatedRecords);
|
||||
metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty());
|
||||
writeClient.commit(newCommitTime, jsc.emptyRDD(), Option.empty());
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
// Verify that all data file has one log file
|
||||
table = HoodieSparkTable.create(config, context);
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<FileSlice> groupedLogFiles =
|
||||
table.getSliceView().getLatestFileSlices(partitionPath).collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : groupedLogFiles) {
|
||||
assertEquals(1, fileSlice.getLogFiles().count(), "There should be 1 log file written for every data file");
|
||||
}
|
||||
}
|
||||
createDeltaCommit(basePath, newCommitTime);
|
||||
createRequestedDeltaCommit(basePath, newCommitTime);
|
||||
createInflightDeltaCommit(basePath, newCommitTime);
|
||||
|
||||
// Do a compaction
|
||||
table = HoodieSparkTable.create(config, context);
|
||||
String compactionInstantTime = "102";
|
||||
table.scheduleCompaction(context, compactionInstantTime, Option.empty());
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
JavaRDD<WriteStatus> result = (JavaRDD<WriteStatus>) table.compact(context, compactionInstantTime).getWriteStatuses();
|
||||
|
||||
// Verify that all partition paths are present in the WriteStatus result
|
||||
for (String partitionPath : dataGen.getPartitionPaths()) {
|
||||
List<WriteStatus> writeStatuses = result.collect();
|
||||
assertTrue(writeStatuses.stream()
|
||||
.filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)).count() > 0);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,116 @@
|
||||
/*
|
||||
* 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.table.action.compact;
|
||||
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class TestInlineCompaction extends CompactionTestBase {
|
||||
|
||||
private HoodieWriteConfig getConfigForInlineCompaction(int maxDeltaCommits) {
|
||||
return getConfigBuilder(false)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withInlineCompaction(true).withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommits).build())
|
||||
.build();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionIsNotScheduledEarly() throws Exception {
|
||||
// Given: make two commits
|
||||
HoodieWriteConfig cfg = getConfigForInlineCompaction(3);
|
||||
try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
|
||||
List<HoodieRecord> records = dataGen.generateInserts("000", 100);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
runNextDeltaCommits(writeClient, readClient, Arrays.asList("000", "001"), records, cfg, true, new ArrayList<>());
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
|
||||
// Then: ensure no compaction is executedm since there are only 2 delta commits
|
||||
assertEquals(2, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSuccessfulCompaction() throws Exception {
|
||||
// Given: make three commits
|
||||
HoodieWriteConfig cfg = getConfigForInlineCompaction(3);
|
||||
List<String> instants = IntStream.range(0, 2).mapToObj(i -> HoodieActiveTimeline.createNewInstantTime()).collect(Collectors.toList());
|
||||
|
||||
try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
|
||||
|
||||
// third commit, that will trigger compaction
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
String finalInstant = HoodieActiveTimeline.createNewInstantTime();
|
||||
createNextDeltaCommit(finalInstant, dataGen.generateUpdates(finalInstant, 100), writeClient, metaClient, cfg, false);
|
||||
|
||||
// Then: ensure the file slices are compacted as per policy
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
|
||||
assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompactionRetryOnFailure() throws Exception {
|
||||
// Given: two commits, schedule compaction and its failed/in-flight
|
||||
HoodieWriteConfig cfg = getConfigBuilder(false)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1).build())
|
||||
.build();
|
||||
List<String> instants = CollectionUtils.createImmutableList("000", "001");
|
||||
try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(cfg)) {
|
||||
List<HoodieRecord> records = dataGen.generateInserts(instants.get(0), 100);
|
||||
HoodieReadClient readClient = getHoodieReadClient(cfg.getBasePath());
|
||||
runNextDeltaCommits(writeClient, readClient, instants, records, cfg, true, new ArrayList<>());
|
||||
// Schedule compaction 002, make it in-flight (simulates inline compaction failing)
|
||||
scheduleCompaction("002", writeClient, cfg);
|
||||
moveCompactionFromRequestedToInflight("002", cfg);
|
||||
}
|
||||
|
||||
// When: a third commit happens
|
||||
HoodieWriteConfig inlineCfg = getConfigForInlineCompaction(2);
|
||||
try (SparkRDDWriteClient<?> writeClient = getHoodieWriteClient(inlineCfg)) {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
createNextDeltaCommit("003", dataGen.generateUpdates("003", 100), writeClient, metaClient, inlineCfg, false);
|
||||
}
|
||||
|
||||
// Then: 1 delta commit is done, the failed compaction is retried
|
||||
metaClient = new HoodieTableMetaClient(hadoopConf, cfg.getBasePath());
|
||||
assertEquals(4, metaClient.getActiveTimeline().getCommitsAndCompactionTimeline().countInstants());
|
||||
assertEquals("002", metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants().firstInstant().get().getTimestamp());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,327 @@
|
||||
/*
|
||||
* 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.table.action.compact.strategy;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionOperation;
|
||||
import org.apache.hudi.common.model.BaseFile;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestHoodieCompactionStrategy {
|
||||
|
||||
private static final long MB = 1024 * 1024L;
|
||||
private String[] partitionPaths = {"2017/01/01", "2017/01/02", "2017/01/03"};
|
||||
private static final Random RANDOM = new Random();
|
||||
|
||||
@Test
|
||||
public void testUnBounded() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
UnBoundedCompactionStrategy strategy = new UnBoundedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp")
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).build()).build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
assertEquals(operations, returned, "UnBounded should not re-order or filter");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBoundedIOSimple() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
BoundedIOCompactionStrategy strategy = new BoundedIOCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
|
||||
.build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue(returned.size() < operations.size(), "BoundedIOCompaction should have resulted in fewer compactions");
|
||||
assertEquals(2, returned.size(), "BoundedIOCompaction should have resulted in 2 compactions being chosen");
|
||||
// Total size of all the log files
|
||||
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
|
||||
.map(Double::longValue).reduce(Long::sum).orElse(0L);
|
||||
assertEquals(610, (long) returnedSize,
|
||||
"Should chose the first 2 compactions which should result in a total IO of 690 MB");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLogFileSizeCompactionSimple() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
LogFileSizeBasedCompactionStrategy strategy = new LogFileSizeBasedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder().withCompactionStrategy(strategy).withTargetIOPerCompactionInMB(400).build())
|
||||
.build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue(returned.size() < operations.size(),
|
||||
"LogFileSizeBasedCompactionStrategy should have resulted in fewer compactions");
|
||||
assertEquals(1, returned.size(), "LogFileSizeBasedCompactionStrategy should have resulted in 1 compaction");
|
||||
// Total size of all the log files
|
||||
Long returnedSize = returned.stream().map(s -> s.getMetrics().get(BoundedIOCompactionStrategy.TOTAL_IO_MB))
|
||||
.map(Double::longValue).reduce(Long::sum).orElse(0L);
|
||||
assertEquals(1204, (long) returnedSize,
|
||||
"Should chose the first 2 compactions which should result in a total IO of 690 MB");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDayBasedCompactionSimple() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, partitionPaths[2]);
|
||||
put(110 * MB, partitionPaths[2]);
|
||||
put(100 * MB, partitionPaths[1]);
|
||||
put(90 * MB, partitionPaths[0]);
|
||||
}
|
||||
});
|
||||
|
||||
DayBasedCompactionStrategy strategy = new DayBasedCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCompactionStrategy(strategy).withTargetPartitionsPerDayBasedCompaction(1).build()).build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue(returned.size() < operations.size(),
|
||||
"DayBasedCompactionStrategy should have resulted in fewer compactions");
|
||||
assertEquals(2, returned.size(), "DayBasedCompactionStrategy should have resulted in fewer compactions");
|
||||
|
||||
int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1).getPartitionPath(),
|
||||
returned.get(0).getPartitionPath());
|
||||
// Either the partition paths are sorted in descending order or they are equal
|
||||
assertTrue(comparision >= 0, "DayBasedCompactionStrategy should sort partitions in descending order");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testBoundedPartitionAwareCompactionSimple() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(70 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(80 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
|
||||
Date today = new Date();
|
||||
String currentDay = format.format(today);
|
||||
|
||||
String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
|
||||
String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
|
||||
String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, currentDay);
|
||||
put(110 * MB, currentDayMinus1);
|
||||
put(100 * MB, currentDayMinus2);
|
||||
put(80 * MB, currentDayMinus3);
|
||||
put(90 * MB, currentDayPlus1);
|
||||
put(70 * MB, currentDayPlus5);
|
||||
}
|
||||
});
|
||||
|
||||
BoundedPartitionAwareCompactionStrategy strategy = new BoundedPartitionAwareCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCompactionStrategy(strategy).withTargetPartitionsPerDayBasedCompaction(2).build()).build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue(returned.size() < operations.size(),
|
||||
"BoundedPartitionAwareCompactionStrategy should have resulted in fewer compactions");
|
||||
assertEquals(5, returned.size(),
|
||||
"BoundedPartitionAwareCompactionStrategy should have resulted in fewer compactions");
|
||||
|
||||
int comparision = strategy.getComparator().compare(returned.get(returned.size() - 1).getPartitionPath(),
|
||||
returned.get(0).getPartitionPath());
|
||||
// Either the partition paths are sorted in descending order or they are equal
|
||||
assertTrue(comparision >= 0, "BoundedPartitionAwareCompactionStrategy should sort partitions in descending order");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUnboundedPartitionAwareCompactionSimple() {
|
||||
Map<Long, List<Long>> sizesMap = new HashMap<>();
|
||||
sizesMap.put(120 * MB, Arrays.asList(60 * MB, 10 * MB, 80 * MB));
|
||||
sizesMap.put(110 * MB, new ArrayList<>());
|
||||
sizesMap.put(100 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(80 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(70 * MB, Collections.singletonList(MB));
|
||||
sizesMap.put(90 * MB, Collections.singletonList(1024 * MB));
|
||||
|
||||
SimpleDateFormat format = new SimpleDateFormat("yyyy/MM/dd");
|
||||
Date today = new Date();
|
||||
String currentDay = format.format(today);
|
||||
|
||||
String currentDayMinus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-1));
|
||||
String currentDayMinus2 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-2));
|
||||
String currentDayMinus3 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(-3));
|
||||
String currentDayPlus1 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(1));
|
||||
String currentDayPlus5 = format.format(BoundedPartitionAwareCompactionStrategy.getDateAtOffsetFromToday(5));
|
||||
|
||||
Map<Long, String> keyToPartitionMap = Collections.unmodifiableMap(new HashMap<Long,String>() {
|
||||
{
|
||||
put(120 * MB, currentDay);
|
||||
put(110 * MB, currentDayMinus1);
|
||||
put(100 * MB, currentDayMinus2);
|
||||
put(80 * MB, currentDayMinus3);
|
||||
put(90 * MB, currentDayPlus1);
|
||||
put(70 * MB, currentDayPlus5);
|
||||
}
|
||||
});
|
||||
|
||||
UnBoundedPartitionAwareCompactionStrategy strategy = new UnBoundedPartitionAwareCompactionStrategy();
|
||||
HoodieWriteConfig writeConfig =
|
||||
HoodieWriteConfig.newBuilder().withPath("/tmp").withCompactionConfig(HoodieCompactionConfig.newBuilder()
|
||||
.withCompactionStrategy(strategy).withTargetPartitionsPerDayBasedCompaction(2).build()).build();
|
||||
List<HoodieCompactionOperation> operations = createCompactionOperations(writeConfig, sizesMap, keyToPartitionMap);
|
||||
List<HoodieCompactionOperation> returned = strategy.orderAndFilter(writeConfig, operations, new ArrayList<>());
|
||||
|
||||
assertTrue(returned.size() < operations.size(),
|
||||
"UnBoundedPartitionAwareCompactionStrategy should not include last "
|
||||
+ writeConfig.getTargetPartitionsPerDayBasedCompaction() + " partitions or later partitions from today");
|
||||
assertEquals(1, returned.size(),
|
||||
"BoundedPartitionAwareCompactionStrategy should have resulted in 1 compaction");
|
||||
}
|
||||
|
||||
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
|
||||
Map<Long, List<Long>> sizesMap) {
|
||||
Map<Long, String> keyToPartitionMap = sizesMap.keySet().stream()
|
||||
.map(e -> Pair.of(e, partitionPaths[RANDOM.nextInt(partitionPaths.length - 1)]))
|
||||
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
|
||||
return createCompactionOperations(config, sizesMap, keyToPartitionMap);
|
||||
}
|
||||
|
||||
private List<HoodieCompactionOperation> createCompactionOperations(HoodieWriteConfig config,
|
||||
Map<Long, List<Long>> sizesMap, Map<Long, String> keyToPartitionMap) {
|
||||
List<HoodieCompactionOperation> operations = new ArrayList<>(sizesMap.size());
|
||||
|
||||
sizesMap.forEach((k, v) -> {
|
||||
HoodieBaseFile df = TestHoodieBaseFile.newDataFile(k);
|
||||
String partitionPath = keyToPartitionMap.get(k);
|
||||
List<HoodieLogFile> logFiles = v.stream().map(TestHoodieLogFile::newLogFile).collect(Collectors.toList());
|
||||
operations.add(new HoodieCompactionOperation(df.getCommitTime(),
|
||||
logFiles.stream().map(s -> s.getPath().toString()).collect(Collectors.toList()), df.getPath(), df.getFileId(),
|
||||
partitionPath,
|
||||
config.getCompactionStrategy().captureMetrics(config, Option.of(df), partitionPath, logFiles),
|
||||
df.getBootstrapBaseFile().map(BaseFile::getPath).orElse(null))
|
||||
);
|
||||
});
|
||||
return operations;
|
||||
}
|
||||
|
||||
public static class TestHoodieBaseFile extends HoodieBaseFile {
|
||||
|
||||
private final long size;
|
||||
|
||||
public TestHoodieBaseFile(long size) {
|
||||
super("/tmp/XYXYXYXYXYYX_11_20180918020003.parquet");
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public static HoodieBaseFile newDataFile(long size) {
|
||||
return new TestHoodieBaseFile(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPath() {
|
||||
return "/tmp/test";
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getFileId() {
|
||||
return UUID.randomUUID().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCommitTime() {
|
||||
return "100";
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return size;
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestHoodieLogFile extends HoodieLogFile {
|
||||
|
||||
private final long size;
|
||||
|
||||
public TestHoodieLogFile(long size) {
|
||||
super("/tmp/.ce481ee7-9e53-4a2e-9992-f9e295fa79c0_20180919184844.log.1");
|
||||
this.size = size;
|
||||
}
|
||||
|
||||
public static HoodieLogFile newLogFile(long size) {
|
||||
return new TestHoodieLogFile(size);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getPath() {
|
||||
return new Path("/tmp/test-log");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileSize() {
|
||||
return size;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.testutils.Assertions;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
|
||||
public class HoodieClientRollbackTestBase extends HoodieClientTestBase {
|
||||
protected void twoUpsertCommitDataWithTwoPartitions(List<FileSlice> firstPartitionCommit2FileSlices,
|
||||
List<FileSlice> secondPartitionCommit2FileSlices,
|
||||
HoodieWriteConfig cfg,
|
||||
boolean commitSecondUpsert) throws IOException {
|
||||
//just generate two partitions
|
||||
dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH});
|
||||
//1. prepare data
|
||||
HoodieTestDataGenerator.writePartitionMetadata(fs, new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, basePath);
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 2);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(statuses.collect());
|
||||
client.commit(newCommitTime, statuses);
|
||||
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
records = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records, 1), newCommitTime);
|
||||
Assertions.assertNoWriteErrors(statuses.collect());
|
||||
if (commitSecondUpsert) {
|
||||
client.commit(newCommitTime, statuses);
|
||||
}
|
||||
|
||||
|
||||
//2. assert filegroup and get the first partition fileslice
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
SyncableFileSystemView fsView = getFileSystemViewWithUnCommittedSlices(table.getMetaClient());
|
||||
List<HoodieFileGroup> firstPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionCommit2FileGroups.size());
|
||||
firstPartitionCommit2FileSlices.addAll(firstPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList()));
|
||||
//3. assert filegroup and get the second partition fileslice
|
||||
List<HoodieFileGroup> secondPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionCommit2FileGroups.size());
|
||||
secondPartitionCommit2FileSlices.addAll(secondPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList()));
|
||||
|
||||
//4. assert fileslice
|
||||
HoodieTableType tableType = this.getTableType();
|
||||
if (tableType.equals(HoodieTableType.COPY_ON_WRITE)) {
|
||||
assertEquals(2, firstPartitionCommit2FileSlices.size());
|
||||
assertEquals(2, secondPartitionCommit2FileSlices.size());
|
||||
} else {
|
||||
assertEquals(1, firstPartitionCommit2FileSlices.size());
|
||||
assertEquals(1, secondPartitionCommit2FileSlices.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,200 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieFileGroup;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
import static org.junit.jupiter.api.Assertions.fail;
|
||||
|
||||
public class TestCopyOnWriteRollbackActionExecutor extends HoodieClientRollbackTestBase {
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
initFileSystem();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() throws Exception {
|
||||
final String p1 = "2015/03/16";
|
||||
final String p2 = "2015/03/17";
|
||||
final String p3 = "2016/03/15";
|
||||
// Let's create some commit files and parquet files
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient)
|
||||
.withPartitionMetaFiles(p1, p2, p3)
|
||||
.addCommit("001")
|
||||
.withBaseFilesInPartition(p1, "id11")
|
||||
.withBaseFilesInPartition(p2, "id12")
|
||||
.withLogFile(p1, "id11", 3)
|
||||
.addCommit("002")
|
||||
.withBaseFilesInPartition(p1, "id21")
|
||||
.withBaseFilesInPartition(p2, "id22");
|
||||
|
||||
HoodieTable table = this.getHoodieTable(metaClient, getConfig());
|
||||
HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002");
|
||||
|
||||
// execute CopyOnWriteRollbackActionExecutor with filelisting mode
|
||||
SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, table.getConfig(), table, "003", needRollBackInstant, true);
|
||||
assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
|
||||
List<HoodieRollbackStat> hoodieRollbackStats = copyOnWriteRollbackActionExecutor.executeRollback();
|
||||
|
||||
// assert hoodieRollbackStats
|
||||
assertEquals(hoodieRollbackStats.size(), 3);
|
||||
for (HoodieRollbackStat stat : hoodieRollbackStats) {
|
||||
switch (stat.getPartitionPath()) {
|
||||
case p1:
|
||||
assertEquals(1, stat.getSuccessDeleteFiles().size());
|
||||
assertEquals(0, stat.getFailedDeleteFiles().size());
|
||||
assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
|
||||
assertEquals(testTable.forCommit("002").getBaseFilePath(p1, "id21").toString(),
|
||||
stat.getSuccessDeleteFiles().get(0));
|
||||
break;
|
||||
case p2:
|
||||
assertEquals(1, stat.getSuccessDeleteFiles().size());
|
||||
assertEquals(0, stat.getFailedDeleteFiles().size());
|
||||
assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
|
||||
assertEquals(testTable.forCommit("002").getBaseFilePath(p2, "id22").toString(),
|
||||
stat.getSuccessDeleteFiles().get(0));
|
||||
break;
|
||||
case p3:
|
||||
assertEquals(0, stat.getSuccessDeleteFiles().size());
|
||||
assertEquals(0, stat.getFailedDeleteFiles().size());
|
||||
assertEquals(Collections.EMPTY_MAP, stat.getCommandBlocksCount());
|
||||
break;
|
||||
default:
|
||||
fail("Unexpected partition: " + stat.getPartitionPath());
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(testTable.inflightCommitExists("001"));
|
||||
assertTrue(testTable.commitExists("001"));
|
||||
assertTrue(testTable.baseFileExists(p1, "001", "id11"));
|
||||
assertTrue(testTable.baseFileExists(p2, "001", "id12"));
|
||||
assertFalse(testTable.inflightCommitExists("002"));
|
||||
assertFalse(testTable.commitExists("002"));
|
||||
assertFalse(testTable.baseFileExists(p1, "002", "id21"));
|
||||
assertFalse(testTable.baseFileExists(p2, "002", "id22"));
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testCopyOnWriteRollbackActionExecutor(boolean isUsingMarkers) throws IOException {
|
||||
//1. prepare data and assert data result
|
||||
List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
|
||||
List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withRollbackUsingMarkers(isUsingMarkers).withAutoCommit(false).build();
|
||||
this.twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, !isUsingMarkers);
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
|
||||
//2. rollback
|
||||
HoodieInstant commitInstant;
|
||||
if (isUsingMarkers) {
|
||||
commitInstant = table.getActiveTimeline().getCommitTimeline().filterInflights().lastInstant().get();
|
||||
} else {
|
||||
commitInstant = table.getCompletedCommitTimeline().lastInstant().get();
|
||||
}
|
||||
|
||||
SparkCopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = new SparkCopyOnWriteRollbackActionExecutor(context, cfg, table, "003", commitInstant, false);
|
||||
if (!isUsingMarkers) {
|
||||
assertFalse(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
|
||||
} else {
|
||||
assertTrue(copyOnWriteRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
|
||||
}
|
||||
Map<String, HoodieRollbackPartitionMetadata> rollbackMetadata = copyOnWriteRollbackActionExecutor.execute().getPartitionMetadata();
|
||||
|
||||
//3. assert the rollback stat
|
||||
assertEquals(2, rollbackMetadata.size());
|
||||
for (Map.Entry<String, HoodieRollbackPartitionMetadata> entry : rollbackMetadata.entrySet()) {
|
||||
HoodieRollbackPartitionMetadata meta = entry.getValue();
|
||||
assertTrue(meta.getFailedDeleteFiles() == null
|
||||
|| meta.getFailedDeleteFiles().size() == 0);
|
||||
assertTrue(meta.getSuccessDeleteFiles() == null
|
||||
|| meta.getSuccessDeleteFiles().size() == 1);
|
||||
}
|
||||
|
||||
//4. assert filegroup after rollback, and compare to the rollbackstat
|
||||
// assert the first partition file group and file slice
|
||||
List<HoodieFileGroup> firstPartitionRollBack1FileGroups = table.getFileSystemView().getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionRollBack1FileGroups.size());
|
||||
List<FileSlice> firstPartitionRollBack1FileSlices = firstPartitionRollBack1FileGroups.get(0).getAllFileSlices().collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionRollBack1FileSlices.size());
|
||||
|
||||
if (!isUsingMarkers) {
|
||||
firstPartitionCommit2FileSlices.removeAll(firstPartitionRollBack1FileSlices);
|
||||
assertEquals(1, firstPartitionCommit2FileSlices.size());
|
||||
assertEquals(firstPartitionCommit2FileSlices.get(0).getBaseFile().get().getPath(),
|
||||
rollbackMetadata.get(DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().get(0));
|
||||
} else {
|
||||
assertEquals(firstPartitionCommit2FileSlices.get(0).getBaseFile().get().getPath(),
|
||||
String.format("%s:%s/%s", this.fs.getScheme(), basePath, rollbackMetadata.get(DEFAULT_FIRST_PARTITION_PATH).getSuccessDeleteFiles().get(0)));
|
||||
}
|
||||
|
||||
// assert the second partition file group and file slice
|
||||
List<HoodieFileGroup> secondPartitionRollBack1FileGroups = table.getFileSystemView().getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionRollBack1FileGroups.size());
|
||||
List<FileSlice> secondPartitionRollBack1FileSlices = secondPartitionRollBack1FileGroups.get(0).getAllFileSlices().collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionRollBack1FileSlices.size());
|
||||
|
||||
// assert the second partition rollback file is equals rollBack1SecondPartitionStat
|
||||
if (!isUsingMarkers) {
|
||||
secondPartitionCommit2FileSlices.removeAll(secondPartitionRollBack1FileSlices);
|
||||
assertEquals(1, secondPartitionCommit2FileSlices.size());
|
||||
assertEquals(secondPartitionCommit2FileSlices.get(0).getBaseFile().get().getPath(),
|
||||
rollbackMetadata.get(DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().get(0));
|
||||
} else {
|
||||
assertEquals(secondPartitionCommit2FileSlices.get(0).getBaseFile().get().getPath(),
|
||||
String.format("%s:%s/%s", this.fs.getScheme(), basePath, rollbackMetadata.get(DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().get(0)));
|
||||
}
|
||||
|
||||
assertFalse(new MarkerFiles(table, commitInstant.getTimestamp()).doesMarkerDirExist());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,126 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.IOType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestMarkerBasedRollbackStrategy extends HoodieClientTestBase {
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
initFileSystem();
|
||||
initMetaClient();
|
||||
initDFS();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyOnWriteRollback() throws Exception {
|
||||
// given: wrote some base files and corresponding markers
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient);
|
||||
String f0 = testTable.addRequestedCommit("000")
|
||||
.withBaseFilesInPartitions("partA").get("partA");
|
||||
String f1 = testTable.addCommit("001")
|
||||
.withBaseFilesInPartition("partA", f0)
|
||||
.withBaseFilesInPartitions("partB").get("partB");
|
||||
String f2 = "f2";
|
||||
testTable.forCommit("001")
|
||||
.withMarkerFile("partA", f0, IOType.MERGE)
|
||||
.withMarkerFile("partB", f1, IOType.CREATE)
|
||||
.withMarkerFile("partA", f2, IOType.CREATE);
|
||||
|
||||
// when
|
||||
List<HoodieRollbackStat> stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002")
|
||||
.execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"));
|
||||
|
||||
// then: ensure files are deleted correctly, non-existent files reported as failed deletes
|
||||
assertEquals(2, stats.size());
|
||||
|
||||
List<FileStatus> partAFiles = testTable.listAllFiles("partA");
|
||||
List<FileStatus> partBFiles = testTable.listAllFiles("partB");
|
||||
|
||||
assertEquals(0, partBFiles.size());
|
||||
assertEquals(1, partAFiles.size());
|
||||
assertEquals(2, stats.stream().mapToInt(r -> r.getSuccessDeleteFiles().size()).sum());
|
||||
assertEquals(1, stats.stream().mapToInt(r -> r.getFailedDeleteFiles().size()).sum());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeOnReadRollback() throws Exception {
|
||||
// given: wrote some base + log files and corresponding markers
|
||||
HoodieTestTable testTable = HoodieTestTable.of(metaClient);
|
||||
String f2 = testTable.addRequestedDeltaCommit("000")
|
||||
.withBaseFilesInPartitions("partA").get("partA");
|
||||
String f1 = testTable.addDeltaCommit("001")
|
||||
.withLogFile("partA", f2)
|
||||
.withBaseFilesInPartitions("partB").get("partB");
|
||||
String f3 = "f3";
|
||||
String f4 = "f4";
|
||||
testTable.forDeltaCommit("001")
|
||||
.withMarkerFile("partB", f1, IOType.CREATE)
|
||||
.withMarkerFile("partA", f3, IOType.CREATE)
|
||||
.withMarkerFile("partA", f2, IOType.APPEND)
|
||||
.withMarkerFile("partB", f4, IOType.APPEND);
|
||||
|
||||
// when
|
||||
List<HoodieRollbackStat> stats = new SparkMarkerBasedRollbackStrategy(HoodieSparkTable.create(getConfig(), context, metaClient), context, getConfig(), "002")
|
||||
.execute(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"));
|
||||
|
||||
// then: ensure files are deleted, rollback block is appended (even if append does not exist)
|
||||
assertEquals(2, stats.size());
|
||||
// will have the log file
|
||||
List<FileStatus> partBFiles = testTable.listAllFiles("partB");
|
||||
assertEquals(1, partBFiles.size());
|
||||
assertTrue(partBFiles.get(0).getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension()));
|
||||
assertTrue(partBFiles.get(0).getLen() > 0);
|
||||
|
||||
List<FileStatus> partAFiles = testTable.listAllFiles("partA");
|
||||
assertEquals(3, partAFiles.size());
|
||||
assertEquals(2, partAFiles.stream().filter(s -> s.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())).count());
|
||||
assertEquals(1, partAFiles.stream().filter(s -> s.getPath().getName().contains(HoodieFileFormat.HOODIE_LOG.getFileExtension())).filter(f -> f.getLen() > 0).count());
|
||||
|
||||
// only partB/f1_001 will be deleted
|
||||
assertEquals(1, stats.stream().mapToInt(r -> r.getSuccessDeleteFiles().size()).sum());
|
||||
// partA/f3_001 is non existent
|
||||
assertEquals(1, stats.stream().mapToInt(r -> r.getFailedDeleteFiles().size()).sum());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
|
||||
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.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.Assertions;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.ValueSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackTestBase {
|
||||
@Override
|
||||
protected HoodieTableType getTableType() {
|
||||
return HoodieTableType.MERGE_ON_READ;
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
//just generate tow partitions
|
||||
dataGen = new HoodieTestDataGenerator(new String[]{DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH});
|
||||
initFileSystem();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = {true, false})
|
||||
public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws IOException {
|
||||
//1. prepare data and assert data result
|
||||
List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
|
||||
List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withRollbackUsingMarkers(isUsingMarkers).withAutoCommit(false).build();
|
||||
twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, !isUsingMarkers);
|
||||
List<HoodieLogFile> firstPartitionCommit2LogFiles = new ArrayList<>();
|
||||
List<HoodieLogFile> secondPartitionCommit2LogFiles = new ArrayList<>();
|
||||
firstPartitionCommit2FileSlices.get(0).getLogFiles().collect(Collectors.toList()).forEach(logFile -> firstPartitionCommit2LogFiles.add(logFile));
|
||||
assertEquals(1, firstPartitionCommit2LogFiles.size());
|
||||
secondPartitionCommit2FileSlices.get(0).getLogFiles().collect(Collectors.toList()).forEach(logFile -> secondPartitionCommit2LogFiles.add(logFile));
|
||||
assertEquals(1, secondPartitionCommit2LogFiles.size());
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
|
||||
//2. rollback
|
||||
HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002");
|
||||
SparkMergeOnReadRollbackActionExecutor mergeOnReadRollbackActionExecutor = new SparkMergeOnReadRollbackActionExecutor(
|
||||
context,
|
||||
cfg,
|
||||
table,
|
||||
"003",
|
||||
rollBackInstant,
|
||||
true);
|
||||
// assert is filelist mode
|
||||
if (!isUsingMarkers) {
|
||||
assertFalse(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
|
||||
} else {
|
||||
assertTrue(mergeOnReadRollbackActionExecutor.getRollbackStrategy() instanceof SparkMarkerBasedRollbackStrategy);
|
||||
}
|
||||
|
||||
//3. assert the rollback stat
|
||||
Map<String, HoodieRollbackPartitionMetadata> rollbackMetadata = mergeOnReadRollbackActionExecutor.execute().getPartitionMetadata();
|
||||
assertEquals(2, rollbackMetadata.size());
|
||||
|
||||
for (Map.Entry<String, HoodieRollbackPartitionMetadata> entry : rollbackMetadata.entrySet()) {
|
||||
HoodieRollbackPartitionMetadata meta = entry.getValue();
|
||||
assertTrue(meta.getFailedDeleteFiles() == null || meta.getFailedDeleteFiles().size() == 0);
|
||||
assertTrue(meta.getSuccessDeleteFiles() == null || meta.getSuccessDeleteFiles().size() == 0);
|
||||
}
|
||||
|
||||
//4. assert filegroup after rollback, and compare to the rollbackstat
|
||||
// assert the first partition data and log file size
|
||||
List<HoodieFileGroup> firstPartitionRollBack1FileGroups = table.getFileSystemView().getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionRollBack1FileGroups.size());
|
||||
List<FileSlice> firstPartitionRollBack1FileSlices = firstPartitionRollBack1FileGroups.get(0).getAllFileSlices().collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionRollBack1FileSlices.size());
|
||||
FileSlice firstPartitionRollBack1FileSlice = firstPartitionRollBack1FileSlices.get(0);
|
||||
List<HoodieLogFile> firstPartitionRollBackLogFiles = firstPartitionRollBack1FileSlice.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals(2, firstPartitionRollBackLogFiles.size());
|
||||
|
||||
firstPartitionRollBackLogFiles.removeAll(firstPartitionCommit2LogFiles);
|
||||
assertEquals(1, firstPartitionRollBackLogFiles.size());
|
||||
|
||||
// assert the second partition data and log file size
|
||||
List<HoodieFileGroup> secondPartitionRollBack1FileGroups = table.getFileSystemView().getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionRollBack1FileGroups.size());
|
||||
List<FileSlice> secondPartitionRollBack1FileSlices = secondPartitionRollBack1FileGroups.get(0).getAllFileSlices().collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionRollBack1FileSlices.size());
|
||||
FileSlice secondPartitionRollBack1FileSlice = secondPartitionRollBack1FileSlices.get(0);
|
||||
List<HoodieLogFile> secondPartitionRollBackLogFiles = secondPartitionRollBack1FileSlice.getLogFiles().collect(Collectors.toList());
|
||||
assertEquals(2, secondPartitionRollBackLogFiles.size());
|
||||
|
||||
secondPartitionRollBackLogFiles.removeAll(secondPartitionCommit2LogFiles);
|
||||
assertEquals(1, secondPartitionRollBackLogFiles.size());
|
||||
|
||||
assertFalse(new MarkerFiles(table, "002").doesMarkerDirExist());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testFailForCompletedInstants() {
|
||||
Assertions.assertThrows(IllegalArgumentException.class, () -> {
|
||||
HoodieInstant rollBackInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "002");
|
||||
new SparkMergeOnReadRollbackActionExecutor(
|
||||
context,
|
||||
getConfigBuilder().build(),
|
||||
getHoodieTable(metaClient, getConfigBuilder().build()),
|
||||
"003",
|
||||
rollBackInstant,
|
||||
true,
|
||||
true,
|
||||
true);
|
||||
});
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,120 @@
|
||||
/*
|
||||
* 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.table.action.rollback;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hudi.common.HoodieRollbackStat;
|
||||
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertIterableEquals;
|
||||
|
||||
public class TestRollbackUtils {
|
||||
|
||||
private FileStatus generateFileStatus(String filePath) {
|
||||
Path dataFile1Path = new Path(filePath);
|
||||
return new FileStatus(1, true, 1, 1, 1, 1,
|
||||
FsPermission.valueOf("-rw-rw-rw-"), "one", "one", null, dataFile1Path);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGenerateHeader() {
|
||||
HoodieInstant hoodieInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101");
|
||||
String instantToRollback = "1";
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> header = RollbackUtils.generateHeader(instantToRollback, hoodieInstant.getTimestamp());
|
||||
Map<HoodieLogBlock.HeaderMetadataType, String> headerExpect = new HashMap<>(3);
|
||||
headerExpect.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101");
|
||||
headerExpect.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "1");
|
||||
headerExpect.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, "0");
|
||||
assertEquals(header, headerExpect);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMergeRollbackStat() {
|
||||
String partitionPath1 = "/partitionPath1/";
|
||||
String partitionPath2 = "/partitionPath2/";
|
||||
//prepare HoodieRollbackStat for different partition
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat1Files = new HashMap<>();
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile1.parquet"), true);
|
||||
dataFilesOnlyStat1Files.put(generateFileStatus(partitionPath1 + "dataFile2.parquet"), true);
|
||||
HoodieRollbackStat dataFilesOnlyStat1 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath1)
|
||||
.withDeletedFileResults(dataFilesOnlyStat1Files).build();
|
||||
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat2Files = new HashMap<>();
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile1.parquet"), true);
|
||||
dataFilesOnlyStat2Files.put(generateFileStatus(partitionPath2 + "dataFile2.parquet"), true);
|
||||
HoodieRollbackStat dataFilesOnlyStat2 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath2)
|
||||
.withDeletedFileResults(dataFilesOnlyStat1Files).build();
|
||||
//1. test different partitionpath merge
|
||||
assertThrows(IllegalArgumentException.class, () -> {
|
||||
RollbackUtils.mergeRollbackStat(dataFilesOnlyStat1,
|
||||
dataFilesOnlyStat2);
|
||||
}, "different partition rollbackstat merge will failed");
|
||||
|
||||
//prepare HoodieRollbackStat for failed and block append
|
||||
Map<FileStatus, Boolean> dataFilesOnlyStat3Files = new HashMap<>();
|
||||
dataFilesOnlyStat3Files.put(generateFileStatus(partitionPath1 + "dataFile1.log"), true);
|
||||
dataFilesOnlyStat3Files.put(generateFileStatus(partitionPath1 + "dataFile3.parquet"), false);
|
||||
HoodieRollbackStat dataFilesOnlyStat3 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath1)
|
||||
.withDeletedFileResults(dataFilesOnlyStat3Files).build();
|
||||
|
||||
Map<FileStatus, Long> dataFilesOnlyStat4Files = new HashMap<>();
|
||||
dataFilesOnlyStat4Files.put(generateFileStatus(partitionPath1 + "dataFile1.log"), 10L);
|
||||
HoodieRollbackStat dataFilesOnlyStat4 = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath1)
|
||||
.withRollbackBlockAppendResults(dataFilesOnlyStat4Files).build();
|
||||
|
||||
//2. test merge dataFilesOnlyStat1 and dataFilesOnlyStat3
|
||||
HoodieRollbackStat dataFilesOnlyStatMerge1 =
|
||||
RollbackUtils.mergeRollbackStat(dataFilesOnlyStat1, dataFilesOnlyStat3);
|
||||
assertEquals(partitionPath1, dataFilesOnlyStatMerge1.getPartitionPath());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3.parquet"),
|
||||
dataFilesOnlyStatMerge1.getFailedDeleteFiles());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1.parquet",
|
||||
partitionPath1 + "dataFile2.parquet", partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge1.getSuccessDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertEquals(0, dataFilesOnlyStatMerge1.getCommandBlocksCount().size());
|
||||
|
||||
//3. test merge dataFilesOnlyStatMerge1 and dataFilesOnlyStat4
|
||||
HoodieRollbackStat dataFilesOnlyStatMerge2 =
|
||||
RollbackUtils.mergeRollbackStat(dataFilesOnlyStatMerge1, dataFilesOnlyStat4);
|
||||
assertEquals(partitionPath1, dataFilesOnlyStatMerge1.getPartitionPath());
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile3.parquet").stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge2.getFailedDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertIterableEquals(CollectionUtils.createImmutableList(partitionPath1 + "dataFile1.parquet",
|
||||
partitionPath1 + "dataFile2.parquet", partitionPath1 + "dataFile1.log").stream().sorted().collect(Collectors.toList()),
|
||||
dataFilesOnlyStatMerge2.getSuccessDeleteFiles().stream().sorted().collect(Collectors.toList()));
|
||||
assertEquals(CollectionUtils.createImmutableMap(generateFileStatus(partitionPath1 + "dataFile1.log"), 10L),
|
||||
dataFilesOnlyStatMerge2.getCommandBlocksCount());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,417 @@
|
||||
/*
|
||||
* 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.table.upgrade;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
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.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableVersion;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.testutils.Assertions;
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase;
|
||||
import org.apache.hudi.testutils.HoodieClientTestUtils;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.params.ParameterizedTest;
|
||||
import org.junit.jupiter.params.provider.Arguments;
|
||||
import org.junit.jupiter.params.provider.MethodSource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Unit tests {@link SparkUpgradeDowngrade}.
|
||||
*/
|
||||
public class TestUpgradeDowngrade extends HoodieClientTestBase {
|
||||
|
||||
private static final String TEST_NAME_WITH_PARAMS = "[{index}] Test with deletePartialMarkerFiles={0} and TableType = {1}";
|
||||
|
||||
public static Stream<Arguments> configParams() {
|
||||
Object[][] data = new Object[][] {
|
||||
{true, HoodieTableType.COPY_ON_WRITE}, {false, HoodieTableType.COPY_ON_WRITE},
|
||||
{true, HoodieTableType.MERGE_ON_READ}, {false, HoodieTableType.MERGE_ON_READ}
|
||||
};
|
||||
return Stream.of(data).map(Arguments::of);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initSparkContexts();
|
||||
initDFS();
|
||||
initTestDataGenerator();
|
||||
initDFSMetaClient();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLeftOverUpdatedPropFileCleanup() throws IOException {
|
||||
testUpgradeInternal(true, true, HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testUpgrade(boolean deletePartialMarkerFiles, HoodieTableType tableType) throws IOException {
|
||||
testUpgradeInternal(false, deletePartialMarkerFiles, tableType);
|
||||
}
|
||||
|
||||
public void testUpgradeInternal(boolean induceResiduesFromPrevUpgrade, boolean deletePartialMarkerFiles, HoodieTableType tableType) throws IOException {
|
||||
// init config, table and client.
|
||||
Map<String, String> params = new HashMap<>();
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ) {
|
||||
params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name());
|
||||
metaClient = HoodieTestUtils.init(dfs.getConf(), dfsBasePath, HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
// prepare data. Make 2 commits, in which 2nd is not committed.
|
||||
List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
|
||||
List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
|
||||
Pair<List<HoodieRecord>, List<HoodieRecord>> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false);
|
||||
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
||||
|
||||
// delete one of the marker files in 2nd commit if need be.
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
||||
List<String> markerPaths = markerFiles.allMarkerFilePaths();
|
||||
if (deletePartialMarkerFiles) {
|
||||
String toDeleteMarkerFile = markerPaths.get(0);
|
||||
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
||||
markerPaths.remove(toDeleteMarkerFile);
|
||||
}
|
||||
|
||||
// set hoodie.table.version to 0 in hoodie.properties file
|
||||
metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ZERO);
|
||||
|
||||
if (induceResiduesFromPrevUpgrade) {
|
||||
createResidualFile();
|
||||
}
|
||||
|
||||
// should re-create marker files for 2nd commit since its pending.
|
||||
new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ONE, cfg, context, null);
|
||||
|
||||
// assert marker files
|
||||
assertMarkerFilesForUpgrade(table, commitInstant, firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices);
|
||||
|
||||
// verify hoodie.table.version got upgraded
|
||||
assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.ONE.versionCode());
|
||||
assertTableVersionFromPropertyFile(HoodieTableVersion.ONE);
|
||||
|
||||
// trigger 3rd commit with marker based rollback enabled.
|
||||
List<HoodieRecord> thirdBatch = triggerCommit("003", tableType, true);
|
||||
|
||||
// Check the entire dataset has all records only from 1st commit and 3rd commit since 2nd is expected to be rolledback.
|
||||
assertRows(inputRecords.getKey(), thirdBatch);
|
||||
if (induceResiduesFromPrevUpgrade) {
|
||||
assertFalse(dfs.exists(new Path(metaClient.getMetaPath(), SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE)));
|
||||
}
|
||||
}
|
||||
|
||||
@ParameterizedTest(name = TEST_NAME_WITH_PARAMS)
|
||||
@MethodSource("configParams")
|
||||
public void testDowngrade(boolean deletePartialMarkerFiles, HoodieTableType tableType) throws IOException {
|
||||
// init config, table and client.
|
||||
Map<String, String> params = new HashMap<>();
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ) {
|
||||
params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name());
|
||||
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
|
||||
}
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params).build();
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
|
||||
|
||||
// prepare data. Make 2 commits, in which 2nd is not committed.
|
||||
List<FileSlice> firstPartitionCommit2FileSlices = new ArrayList<>();
|
||||
List<FileSlice> secondPartitionCommit2FileSlices = new ArrayList<>();
|
||||
Pair<List<HoodieRecord>, List<HoodieRecord>> inputRecords = twoUpsertCommitDataWithTwoPartitions(firstPartitionCommit2FileSlices, secondPartitionCommit2FileSlices, cfg, client, false);
|
||||
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
HoodieInstant commitInstant = table.getPendingCommitTimeline().lastInstant().get();
|
||||
|
||||
// delete one of the marker files in 2nd commit if need be.
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
||||
List<String> markerPaths = markerFiles.allMarkerFilePaths();
|
||||
if (deletePartialMarkerFiles) {
|
||||
String toDeleteMarkerFile = markerPaths.get(0);
|
||||
table.getMetaClient().getFs().delete(new Path(table.getMetaClient().getTempFolderPath() + "/" + commitInstant.getTimestamp() + "/" + toDeleteMarkerFile));
|
||||
markerPaths.remove(toDeleteMarkerFile);
|
||||
}
|
||||
|
||||
// set hoodie.table.version to 1 in hoodie.properties file
|
||||
prepForDowngrade();
|
||||
|
||||
// downgrade should be performed. all marker files should be deleted
|
||||
new SparkUpgradeDowngrade(metaClient, cfg, context).run(metaClient, HoodieTableVersion.ZERO, cfg, context, null);
|
||||
|
||||
// assert marker files
|
||||
assertMarkerFilesForDowngrade(table, commitInstant);
|
||||
|
||||
// verify hoodie.table.version got downgraded
|
||||
assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.ZERO.versionCode());
|
||||
assertTableVersionFromPropertyFile(HoodieTableVersion.ZERO);
|
||||
|
||||
// trigger 3rd commit with marker based rollback disabled.
|
||||
List<HoodieRecord> thirdBatch = triggerCommit("003", tableType, false);
|
||||
|
||||
// Check the entire dataset has all records only from 1st commit and 3rd commit since 2nd is expected to be rolledback.
|
||||
assertRows(inputRecords.getKey(), thirdBatch);
|
||||
}
|
||||
|
||||
private void assertMarkerFilesForDowngrade(HoodieTable table, HoodieInstant commitInstant) throws IOException {
|
||||
// Verify recreated marker files are as expected
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
||||
assertFalse(markerFiles.doesMarkerDirExist());
|
||||
}
|
||||
|
||||
private void assertMarkerFilesForUpgrade(HoodieTable table, HoodieInstant commitInstant, List<FileSlice> firstPartitionCommit2FileSlices,
|
||||
List<FileSlice> secondPartitionCommit2FileSlices) throws IOException {
|
||||
// Verify recreated marker files are as expected
|
||||
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
|
||||
assertTrue(markerFiles.doesMarkerDirExist());
|
||||
List<String> files = markerFiles.allMarkerFilePaths();
|
||||
|
||||
assertEquals(2, files.size());
|
||||
List<String> actualFiles = new ArrayList<>();
|
||||
for (String file : files) {
|
||||
String fileName = MarkerFiles.stripMarkerSuffix(file);
|
||||
actualFiles.add(fileName);
|
||||
}
|
||||
|
||||
List<FileSlice> expectedFileSlices = new ArrayList<>();
|
||||
expectedFileSlices.addAll(firstPartitionCommit2FileSlices);
|
||||
expectedFileSlices.addAll(secondPartitionCommit2FileSlices);
|
||||
|
||||
List<String> expectedPaths = new ArrayList<>();
|
||||
List<Pair<String, String>> expectedLogFilePaths = new ArrayList<>();
|
||||
|
||||
for (FileSlice fileSlice : expectedFileSlices) {
|
||||
String partitionPath = fileSlice.getPartitionPath();
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
for (HoodieLogFile logFile : fileSlice.getLogFiles().collect(Collectors.toList())) {
|
||||
// log file format can't be matched as is, since the write token can't be asserted. Hence asserting for partitionpath, fileId and baseCommit time.
|
||||
String logBaseCommitTime = logFile.getBaseCommitTime();
|
||||
expectedLogFilePaths.add(Pair.of(partitionPath + "/" + logFile.getFileId(), logBaseCommitTime));
|
||||
}
|
||||
}
|
||||
if (fileSlice.getBaseInstantTime().equals(commitInstant.getTimestamp())) {
|
||||
String path = fileSlice.getBaseFile().get().getPath();
|
||||
// for base files, path can be asserted as is.
|
||||
expectedPaths.add(path.substring(path.indexOf(partitionPath)));
|
||||
}
|
||||
}
|
||||
|
||||
// Trim log file paths only
|
||||
List<String> trimmedActualFiles = new ArrayList<>();
|
||||
for (String actualFile : actualFiles) {
|
||||
if (table.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
|
||||
trimmedActualFiles.add(actualFile.substring(0, actualFile.lastIndexOf('.')));
|
||||
} else {
|
||||
trimmedActualFiles.add(actualFile);
|
||||
}
|
||||
}
|
||||
// assert for base files.
|
||||
for (String expected : expectedPaths) {
|
||||
if (trimmedActualFiles.contains(expected)) {
|
||||
trimmedActualFiles.remove(expected);
|
||||
}
|
||||
}
|
||||
|
||||
if (expectedLogFilePaths.size() > 0) {
|
||||
// assert for log files
|
||||
List<Pair<String, String>> actualLogFiles = new ArrayList<>();
|
||||
for (String actual : trimmedActualFiles) {
|
||||
actualLogFiles.add(Pair.of(actual.substring(0, actual.indexOf('_')), actual.substring(actual.lastIndexOf('_') + 1)));
|
||||
}
|
||||
assertEquals(expectedLogFilePaths.size(), actualLogFiles.size());
|
||||
for (Pair<String, String> entry : expectedLogFilePaths) {
|
||||
assertTrue(actualLogFiles.contains(entry));
|
||||
}
|
||||
} else {
|
||||
assertTrue(trimmedActualFiles.size() == 0);
|
||||
}
|
||||
}
|
||||
|
||||
private List<HoodieRecord> triggerCommit(String newCommitTime, HoodieTableType tableType, boolean enableMarkedBasedRollback) {
|
||||
Map<String, String> params = new HashMap<>();
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ) {
|
||||
params.put(HOODIE_TABLE_TYPE_PROP_NAME, HoodieTableType.MERGE_ON_READ.name());
|
||||
}
|
||||
HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(enableMarkedBasedRollback).withProps(params).build();
|
||||
SparkRDDWriteClient client = getHoodieWriteClient(cfg, true);
|
||||
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 2);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(statuses.collect());
|
||||
client.commit(newCommitTime, statuses);
|
||||
return records;
|
||||
}
|
||||
|
||||
private void assertRows(List<HoodieRecord> firstBatch, List<HoodieRecord> secondBatch) {
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
Dataset<Row> rows = HoodieClientTestUtils.read(jsc, metaClient.getBasePath(), sqlContext, metaClient.getFs(), fullPartitionPaths);
|
||||
List<String> expectedRecordKeys = new ArrayList<>();
|
||||
for (HoodieRecord rec : firstBatch) {
|
||||
expectedRecordKeys.add(rec.getRecordKey());
|
||||
}
|
||||
|
||||
for (HoodieRecord rec : secondBatch) {
|
||||
expectedRecordKeys.add(rec.getRecordKey());
|
||||
}
|
||||
List<Row> rowList = rows.collectAsList();
|
||||
assertEquals(expectedRecordKeys.size(), rows.count());
|
||||
|
||||
List<String> actualRecordKeys = new ArrayList<>();
|
||||
for (Row row : rowList) {
|
||||
actualRecordKeys.add(row.getAs("_row_key"));
|
||||
}
|
||||
|
||||
for (String expectedRecordKey : expectedRecordKeys) {
|
||||
assertTrue(actualRecordKeys.contains(expectedRecordKey));
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create two commits and may or may not commit 2nd commit.
|
||||
*
|
||||
* @param firstPartitionCommit2FileSlices list to hold file slices in first partition.
|
||||
* @param secondPartitionCommit2FileSlices list of hold file slices from second partition.
|
||||
* @param cfg instance of {@link HoodieWriteConfig}
|
||||
* @param client instance of {@link SparkRDDWriteClient} to use.
|
||||
* @param commitSecondUpsert true if 2nd commit needs to be committed. false otherwise.
|
||||
* @return a pair of list of records from 1st and 2nd batch.
|
||||
*/
|
||||
private Pair<List<HoodieRecord>, List<HoodieRecord>> twoUpsertCommitDataWithTwoPartitions(List<FileSlice> firstPartitionCommit2FileSlices,
|
||||
List<FileSlice> secondPartitionCommit2FileSlices,
|
||||
HoodieWriteConfig cfg, SparkRDDWriteClient client,
|
||||
boolean commitSecondUpsert) throws IOException {
|
||||
//just generate two partitions
|
||||
dataGen = new HoodieTestDataGenerator(new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH});
|
||||
//1. prepare data
|
||||
HoodieTestDataGenerator.writePartitionMetadata(dfs, new String[] {DEFAULT_FIRST_PARTITION_PATH, DEFAULT_SECOND_PARTITION_PATH}, dfsBasePath);
|
||||
/**
|
||||
* Write 1 (only inserts)
|
||||
*/
|
||||
String newCommitTime = "001";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
List<HoodieRecord> records = dataGen.generateInsertsContainsAllPartitions(newCommitTime, 2);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
JavaRDD<WriteStatus> statuses = client.upsert(writeRecords, newCommitTime);
|
||||
Assertions.assertNoWriteErrors(statuses.collect());
|
||||
client.commit(newCommitTime, statuses);
|
||||
/**
|
||||
* Write 2 (updates)
|
||||
*/
|
||||
newCommitTime = "002";
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records2 = dataGen.generateUpdates(newCommitTime, records);
|
||||
statuses = client.upsert(jsc.parallelize(records2, 1), newCommitTime);
|
||||
Assertions.assertNoWriteErrors(statuses.collect());
|
||||
if (commitSecondUpsert) {
|
||||
client.commit(newCommitTime, statuses);
|
||||
}
|
||||
|
||||
//2. assert filegroup and get the first partition fileslice
|
||||
HoodieTable table = this.getHoodieTable(metaClient, cfg);
|
||||
SyncableFileSystemView fsView = getFileSystemViewWithUnCommittedSlices(table.getMetaClient());
|
||||
List<HoodieFileGroup> firstPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_FIRST_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, firstPartitionCommit2FileGroups.size());
|
||||
firstPartitionCommit2FileSlices.addAll(firstPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList()));
|
||||
//3. assert filegroup and get the second partition fileslice
|
||||
List<HoodieFileGroup> secondPartitionCommit2FileGroups = fsView.getAllFileGroups(DEFAULT_SECOND_PARTITION_PATH).collect(Collectors.toList());
|
||||
assertEquals(1, secondPartitionCommit2FileGroups.size());
|
||||
secondPartitionCommit2FileSlices.addAll(secondPartitionCommit2FileGroups.get(0).getAllFileSlices().collect(Collectors.toList()));
|
||||
|
||||
//4. assert fileslice
|
||||
HoodieTableType tableType = metaClient.getTableType();
|
||||
if (tableType.equals(HoodieTableType.COPY_ON_WRITE)) {
|
||||
assertEquals(2, firstPartitionCommit2FileSlices.size());
|
||||
assertEquals(2, secondPartitionCommit2FileSlices.size());
|
||||
} else {
|
||||
assertEquals(1, firstPartitionCommit2FileSlices.size());
|
||||
assertEquals(1, secondPartitionCommit2FileSlices.size());
|
||||
}
|
||||
return Pair.of(records, records2);
|
||||
}
|
||||
|
||||
private void prepForDowngrade() throws IOException {
|
||||
metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ONE);
|
||||
Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
|
||||
try (FSDataOutputStream os = metaClient.getFs().create(propertyFile)) {
|
||||
metaClient.getTableConfig().getProperties().store(os, "");
|
||||
}
|
||||
}
|
||||
|
||||
private void createResidualFile() throws IOException {
|
||||
Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
|
||||
Path updatedPropertyFile = new Path(metaClient.getMetaPath() + "/" + SparkUpgradeDowngrade.HOODIE_UPDATED_PROPERTY_FILE);
|
||||
|
||||
// Step1: Copy hoodie.properties to hoodie.properties.orig
|
||||
FileUtil.copy(metaClient.getFs(), propertyFile, metaClient.getFs(), updatedPropertyFile,
|
||||
false, metaClient.getHadoopConf());
|
||||
}
|
||||
|
||||
private void assertTableVersionFromPropertyFile(HoodieTableVersion expectedVersion) throws IOException {
|
||||
Path propertyFile = new Path(metaClient.getMetaPath() + "/" + HoodieTableConfig.HOODIE_PROPERTIES_FILE);
|
||||
// Load the properties and verify
|
||||
FSDataInputStream fsDataInputStream = metaClient.getFs().open(propertyFile);
|
||||
Properties prop = new Properties();
|
||||
prop.load(fsDataInputStream);
|
||||
fsDataInputStream.close();
|
||||
assertEquals(Integer.toString(expectedVersion.versionCode()), prop.getProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,41 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertAll;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
|
||||
/**
|
||||
* Commonly used assertion functions.
|
||||
*/
|
||||
public class Assertions {
|
||||
|
||||
/**
|
||||
* Assert no failures in writing hoodie files.
|
||||
*/
|
||||
public static void assertNoWriteErrors(List<WriteStatus> statuses) {
|
||||
assertAll(statuses.stream().map(status -> () ->
|
||||
assertFalse(status.hasErrors(), "Errors found in write of " + status.getFileId())));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,168 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieEngineContext;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.testutils.providers.DFSProvider;
|
||||
import org.apache.hudi.testutils.providers.HoodieMetaClientProvider;
|
||||
import org.apache.hudi.testutils.providers.HoodieWriteClientProvider;
|
||||
import org.apache.hudi.testutils.providers.SparkProvider;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
import org.junit.jupiter.api.AfterAll;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
|
||||
import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
|
||||
import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME;
|
||||
|
||||
public class FunctionalTestHarness implements SparkProvider, DFSProvider, HoodieMetaClientProvider, HoodieWriteClientProvider {
|
||||
|
||||
private static transient SparkSession spark;
|
||||
private static transient SQLContext sqlContext;
|
||||
private static transient JavaSparkContext jsc;
|
||||
protected static transient HoodieSparkEngineContext context;
|
||||
|
||||
private static transient HdfsTestService hdfsTestService;
|
||||
private static transient MiniDFSCluster dfsCluster;
|
||||
private static transient DistributedFileSystem dfs;
|
||||
|
||||
/**
|
||||
* An indicator of the initialization status.
|
||||
*/
|
||||
protected boolean initialized = false;
|
||||
@TempDir
|
||||
protected java.nio.file.Path tempDir;
|
||||
|
||||
public String basePath() {
|
||||
return tempDir.toAbsolutePath().toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public SparkSession spark() {
|
||||
return spark;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SQLContext sqlContext() {
|
||||
return sqlContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public JavaSparkContext jsc() {
|
||||
return jsc;
|
||||
}
|
||||
|
||||
@Override
|
||||
public MiniDFSCluster dfsCluster() {
|
||||
return dfsCluster;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DistributedFileSystem dfs() {
|
||||
return dfs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path dfsBasePath() {
|
||||
return dfs.getWorkingDirectory();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieEngineContext context() {
|
||||
return context;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath) throws IOException {
|
||||
return getHoodieMetaClient(hadoopConf, basePath, new Properties());
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieTableMetaClient getHoodieMetaClient(Configuration hadoopConf, String basePath, Properties props) throws IOException {
|
||||
props.putIfAbsent(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, PARQUET.toString());
|
||||
props.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, RAW_TRIPS_TEST_NAME);
|
||||
props.putIfAbsent(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, COPY_ON_WRITE.name());
|
||||
props.putIfAbsent(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, HoodieAvroPayload.class.getName());
|
||||
return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, basePath, props);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) throws IOException {
|
||||
return new SparkRDDWriteClient(context(), cfg, false);
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
public synchronized void runBeforeEach() throws Exception {
|
||||
initialized = spark != null && hdfsTestService != null;
|
||||
if (!initialized) {
|
||||
SparkConf sparkConf = conf();
|
||||
SparkRDDWriteClient.registerClasses(sparkConf);
|
||||
HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
spark = SparkSession.builder().config(sparkConf).getOrCreate();
|
||||
sqlContext = spark.sqlContext();
|
||||
jsc = new JavaSparkContext(spark.sparkContext());
|
||||
context = new HoodieSparkEngineContext(jsc);
|
||||
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfs.mkdirs(dfs.getWorkingDirectory());
|
||||
|
||||
Runtime.getRuntime().addShutdownHook(new Thread(() -> {
|
||||
hdfsTestService.stop();
|
||||
hdfsTestService = null;
|
||||
|
||||
spark.stop();
|
||||
spark = null;
|
||||
}));
|
||||
}
|
||||
}
|
||||
|
||||
@AfterAll
|
||||
public static synchronized void cleanUpAfterAll() throws IOException {
|
||||
Path workDir = dfs.getWorkingDirectory();
|
||||
FileSystem fs = workDir.getFileSystem(hdfsTestService.getHadoopConf());
|
||||
FileStatus[] fileStatuses = dfs.listStatus(workDir);
|
||||
for (FileStatus f : fileStatuses) {
|
||||
fs.delete(f.getPath(), true);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,562 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.HoodieCleanStat;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.common.table.view.SyncableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex.IndexType;
|
||||
import org.apache.hudi.index.SparkHoodieIndex;
|
||||
import org.apache.hudi.table.HoodieSparkTable;
|
||||
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors;
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertFalse;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
/**
|
||||
* Base Class providing setup/cleanup and utility methods for testing Hoodie Client facing tests.
|
||||
*/
|
||||
public class HoodieClientTestBase extends HoodieClientTestHarness {
|
||||
|
||||
protected static final Logger LOG = LogManager.getLogger(HoodieClientTestBase.class);
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws Exception {
|
||||
initResources();
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
public void tearDown() throws Exception {
|
||||
cleanupResources();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Default HoodieWriteConfig for tests.
|
||||
*
|
||||
* @return Default Hoodie Write Config for tests
|
||||
*/
|
||||
public HoodieWriteConfig getConfig() {
|
||||
return getConfigBuilder().build();
|
||||
}
|
||||
|
||||
public HoodieWriteConfig getConfig(IndexType indexType) {
|
||||
return getConfigBuilder(indexType).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Config builder with default configs set.
|
||||
*
|
||||
* @return Config Builder
|
||||
*/
|
||||
public HoodieWriteConfig.Builder getConfigBuilder() {
|
||||
return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Config builder with default configs set.
|
||||
*
|
||||
* @return Config Builder
|
||||
*/
|
||||
public HoodieWriteConfig.Builder getConfigBuilder(IndexType indexType) {
|
||||
return getConfigBuilder(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA, indexType);
|
||||
}
|
||||
|
||||
public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) {
|
||||
return getConfigBuilder(schemaStr, IndexType.BLOOM);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Config builder with default configs set.
|
||||
*
|
||||
* @return Config Builder
|
||||
*/
|
||||
public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr, IndexType indexType) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(schemaStr)
|
||||
.withParallelism(2, 2).withBulkInsertParallelism(2).withFinalizeWriteParallelism(2).withDeleteParallelism(2)
|
||||
.withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION)
|
||||
.withWriteStatusClass(MetadataMergeWriteStatus.class)
|
||||
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType).build())
|
||||
.withEmbeddedTimelineServerEnabled(true).withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder()
|
||||
.withEnableBackupForRemoteFileSystemView(false) // Fail test if problem connecting to timeline-server
|
||||
.withStorageType(FileSystemViewStorageType.EMBEDDED_KV_STORE).build());
|
||||
}
|
||||
|
||||
public HoodieSparkTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWriteConfig config) {
|
||||
HoodieSparkTable table = HoodieSparkTable.create(config, context, metaClient);
|
||||
((SyncableFileSystemView) (table.getSliceView())).reset();
|
||||
return table;
|
||||
}
|
||||
|
||||
public void assertPartitionMetadataForRecords(List<HoodieRecord> inputRecords, FileSystem fs) throws IOException {
|
||||
Set<String> partitionPathSet = inputRecords.stream()
|
||||
.map(HoodieRecord::getPartitionPath)
|
||||
.collect(Collectors.toSet());
|
||||
assertPartitionMetadata(partitionPathSet.stream().toArray(String[]::new), fs);
|
||||
}
|
||||
|
||||
public void assertPartitionMetadataForKeys(List<HoodieKey> inputKeys, FileSystem fs) throws IOException {
|
||||
Set<String> partitionPathSet = inputKeys.stream()
|
||||
.map(HoodieKey::getPartitionPath)
|
||||
.collect(Collectors.toSet());
|
||||
assertPartitionMetadata(partitionPathSet.stream().toArray(String[]::new), fs);
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure presence of partition meta-data at known depth.
|
||||
*
|
||||
* @param partitionPaths Partition paths to check
|
||||
* @param fs File System
|
||||
* @throws IOException in case of error
|
||||
*/
|
||||
public void assertPartitionMetadata(String[] partitionPaths, FileSystem fs) throws IOException {
|
||||
for (String partitionPath : partitionPaths) {
|
||||
assertTrue(HoodiePartitionMetadata.hasPartitionMetadata(fs, new Path(basePath, partitionPath)));
|
||||
HoodiePartitionMetadata pmeta = new HoodiePartitionMetadata(fs, new Path(basePath, partitionPath));
|
||||
pmeta.readFromFS();
|
||||
assertEquals(HoodieTestDataGenerator.DEFAULT_PARTITION_DEPTH, pmeta.getPartitionDepth());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure records have location field set.
|
||||
*
|
||||
* @param taggedRecords Tagged Records
|
||||
* @param instantTime Commit Timestamp
|
||||
*/
|
||||
public void checkTaggedRecords(List<HoodieRecord> taggedRecords, String instantTime) {
|
||||
for (HoodieRecord rec : taggedRecords) {
|
||||
assertTrue(rec.isCurrentLocationKnown(), "Record " + rec + " found with no location.");
|
||||
assertEquals(rec.getCurrentLocation().getInstantTime(), instantTime,
|
||||
"All records should have commit time " + instantTime + ", since updates were made");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Assert that there is no duplicate key at the partition level.
|
||||
*
|
||||
* @param records List of Hoodie records
|
||||
*/
|
||||
public void assertNodupesWithinPartition(List<HoodieRecord<RawTripTestPayload>> records) {
|
||||
Map<String, Set<String>> partitionToKeys = new HashMap<>();
|
||||
for (HoodieRecord r : records) {
|
||||
String key = r.getRecordKey();
|
||||
String partitionPath = r.getPartitionPath();
|
||||
if (!partitionToKeys.containsKey(partitionPath)) {
|
||||
partitionToKeys.put(partitionPath, new HashSet<>());
|
||||
}
|
||||
assertFalse(partitionToKeys.get(partitionPath).contains(key), "key " + key + " is duplicate within partition " + partitionPath);
|
||||
partitionToKeys.get(partitionPath).add(key);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to generate records generation function for testing Prepped version of API. Prepped APIs expect the records
|
||||
* to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is
|
||||
* guaranteed by record-generation function itself.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param recordGenFunction Records Generation function
|
||||
* @return Wrapped function
|
||||
*/
|
||||
private Function2<List<HoodieRecord>, String, Integer> wrapRecordsGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function2<List<HoodieRecord>, String, Integer> recordGenFunction) {
|
||||
return (commit, numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
List<HoodieRecord> records = recordGenFunction.apply(commit, numRecords);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(jsc.parallelize(records, 1), context, table);
|
||||
return taggedRecords.collect();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to generate delete keys generation function for testing Prepped version of API. Prepped APIs expect the keys
|
||||
* to be already de-duped and have location set. This wrapper takes care of record-location setting. Uniqueness is
|
||||
* guaranteed by key-generation function itself.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param keyGenFunction Keys Generation function
|
||||
* @return Wrapped function
|
||||
*/
|
||||
private Function<Integer, List<HoodieKey>> wrapDeleteKeysGenFunctionForPreppedCalls(
|
||||
final HoodieWriteConfig writeConfig, final Function<Integer, List<HoodieKey>> keyGenFunction) {
|
||||
return (numRecords) -> {
|
||||
final SparkHoodieIndex index = SparkHoodieIndex.createIndex(writeConfig);
|
||||
List<HoodieKey> records = keyGenFunction.apply(numRecords);
|
||||
final HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath, true);
|
||||
HoodieSparkTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
|
||||
JavaRDD<HoodieRecord> recordsToDelete = jsc.parallelize(records, 1)
|
||||
.map(key -> new HoodieRecord(key, new EmptyHoodieRecordPayload()));
|
||||
JavaRDD<HoodieRecord> taggedRecords = index.tagLocation(recordsToDelete, context, table);
|
||||
return taggedRecords.map(record -> record.getKey()).collect();
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate wrapper for record generation function for testing Prepped APIs.
|
||||
*
|
||||
* @param isPreppedAPI Flag to indicate if this is for testing prepped-version of APIs
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param wrapped Actual Records Generation function
|
||||
* @return Wrapped Function
|
||||
*/
|
||||
public Function2<List<HoodieRecord>, String, Integer> generateWrapRecordsFn(boolean isPreppedAPI,
|
||||
HoodieWriteConfig writeConfig,
|
||||
Function2<List<HoodieRecord>, String, Integer> wrapped) {
|
||||
if (isPreppedAPI) {
|
||||
return wrapRecordsGenFunctionForPreppedCalls(writeConfig, wrapped);
|
||||
} else {
|
||||
return wrapped;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate wrapper for delete key generation function for testing Prepped APIs.
|
||||
*
|
||||
* @param isPreppedAPI Flag to indicate if this is for testing prepped-version of APIs
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param wrapped Actual Records Generation function
|
||||
* @return Wrapped Function
|
||||
*/
|
||||
public Function<Integer, List<HoodieKey>> generateWrapDeleteKeysFn(boolean isPreppedAPI,
|
||||
HoodieWriteConfig writeConfig, Function<Integer, List<HoodieKey>> wrapped) {
|
||||
if (isPreppedAPI) {
|
||||
return wrapDeleteKeysGenFunctionForPreppedCalls(writeConfig, wrapped);
|
||||
} else {
|
||||
return wrapped;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to insert first batch of records and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param writeFn Write Function to be used for insertion
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
|
||||
String initCommitTime, int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
|
||||
boolean assertForCommit, int expRecordsInThisCommit) throws Exception {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
|
||||
|
||||
return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit,
|
||||
recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to insert another batch of records and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param writeFn Write Function to be used for insertion
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @param expTotalCommits Expected number of commits (including this commit)
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
|
||||
String initCommitTime, int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
|
||||
boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts);
|
||||
|
||||
return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit,
|
||||
recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to upsert batch of records and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param writeFn Write Function to be used for upsert
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @param expTotalCommits Expected number of commits (including this commit)
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> updateBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
|
||||
String prevCommitTime, Option<List<String>> commitTimesBetweenPrevAndNew, String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn, boolean isPreppedAPI,
|
||||
boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
|
||||
final Function2<List<HoodieRecord>, String, Integer> recordGenFunction =
|
||||
generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates);
|
||||
|
||||
return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime,
|
||||
numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords,
|
||||
expTotalCommits);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to delete batch of keys and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param writeConfig Hoodie Write Config
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param deleteFn Delete Function to be used for deletes
|
||||
* @param isPreppedAPI Boolean flag to indicate writeFn expects prepped records
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @return RDD of write-status
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime,
|
||||
String prevCommitTime, String initCommitTime,
|
||||
int numRecordsInThisCommit,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieKey>, String> deleteFn, boolean isPreppedAPI,
|
||||
boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception {
|
||||
final Function<Integer, List<HoodieKey>> keyGenFunction =
|
||||
generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes);
|
||||
|
||||
return deleteBatch(client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit,
|
||||
keyGenFunction,
|
||||
deleteFn, assertForCommit, expRecordsInThisCommit, expTotalRecords);
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to insert/upsert batch of records and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param commitTimesBetweenPrevAndNew Sample of Timestamps between prevCommitTime and newCommitTime
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param numRecordsInThisCommit Number of records to be added in the new commit
|
||||
* @param recordGenFunction Records Generation Function
|
||||
* @param writeFn Write Function to be used for upsert
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @param expTotalCommits Expected number of commits (including this commit)
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime,
|
||||
Option<List<String>> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit,
|
||||
Function2<List<HoodieRecord>, String, Integer> recordGenFunction,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieRecord>, String> writeFn,
|
||||
boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception {
|
||||
|
||||
// Write 1 (only inserts)
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieRecord> records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit);
|
||||
JavaRDD<HoodieRecord> writeRecords = jsc.parallelize(records, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = writeFn.apply(client, writeRecords, newCommitTime);
|
||||
List<WriteStatus> statuses = result.collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// check the partition metadata is written out
|
||||
assertPartitionMetadataForRecords(records, fs);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
|
||||
if (assertForCommit) {
|
||||
assertEquals(expTotalCommits, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(),
|
||||
"Expecting " + expTotalCommits + " commits.");
|
||||
assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(),
|
||||
"Latest commit should be " + newCommitTime);
|
||||
assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
"Must contain " + expRecordsInThisCommit + " records");
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals(expTotalRecords, HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(),
|
||||
"Must contain " + expTotalRecords + " records");
|
||||
|
||||
// Check that the incremental consumption from prevCommitTime
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
|
||||
"Incremental consumption from " + prevCommitTime + " should give all records in latest commit");
|
||||
if (commitTimesBetweenPrevAndNew.isPresent()) {
|
||||
commitTimesBetweenPrevAndNew.get().forEach(ct -> {
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, ct),
|
||||
"Incremental consumption from " + ct + " should give all records in latest commit");
|
||||
});
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper to delete batch of hoodie keys and do regular assertions on the state after successful completion.
|
||||
*
|
||||
* @param client Hoodie Write Client
|
||||
* @param newCommitTime New Commit Timestamp to be used
|
||||
* @param prevCommitTime Commit Timestamp used in previous commit
|
||||
* @param initCommitTime Begin Timestamp (usually "000")
|
||||
* @param keyGenFunction Key Generation function
|
||||
* @param deleteFn Write Function to be used for delete
|
||||
* @param assertForCommit Enable Assertion of Writes
|
||||
* @param expRecordsInThisCommit Expected number of records in this commit
|
||||
* @param expTotalRecords Expected number of records when scanned
|
||||
* @throws Exception in case of error
|
||||
*/
|
||||
public JavaRDD<WriteStatus> deleteBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime,
|
||||
String initCommitTime, int numRecordsInThisCommit,
|
||||
Function<Integer, List<HoodieKey>> keyGenFunction,
|
||||
Function3<JavaRDD<WriteStatus>, SparkRDDWriteClient, JavaRDD<HoodieKey>, String> deleteFn,
|
||||
boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords) throws Exception {
|
||||
|
||||
// Delete 1 (only deletes)
|
||||
client.startCommitWithTime(newCommitTime);
|
||||
|
||||
List<HoodieKey> keysToDelete = keyGenFunction.apply(numRecordsInThisCommit);
|
||||
JavaRDD<HoodieKey> deleteRecords = jsc.parallelize(keysToDelete, 1);
|
||||
|
||||
JavaRDD<WriteStatus> result = deleteFn.apply(client, deleteRecords, newCommitTime);
|
||||
List<WriteStatus> statuses = result.collect();
|
||||
assertNoWriteErrors(statuses);
|
||||
|
||||
// check the partition metadata is written out
|
||||
assertPartitionMetadataForKeys(keysToDelete, fs);
|
||||
|
||||
// verify that there is a commit
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(hadoopConf, basePath);
|
||||
HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitTimeline();
|
||||
|
||||
if (assertForCommit) {
|
||||
assertEquals(3, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(),
|
||||
"Expecting 3 commits.");
|
||||
assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(),
|
||||
"Latest commit should be " + newCommitTime);
|
||||
assertEquals(expRecordsInThisCommit, HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
"Must contain " + expRecordsInThisCommit + " records");
|
||||
|
||||
// Check the entire dataset has all records still
|
||||
String[] fullPartitionPaths = new String[dataGen.getPartitionPaths().length];
|
||||
for (int i = 0; i < fullPartitionPaths.length; i++) {
|
||||
fullPartitionPaths[i] = String.format("%s/%s/*", basePath, dataGen.getPartitionPaths()[i]);
|
||||
}
|
||||
assertEquals(expTotalRecords, HoodieClientTestUtils.read(jsc, basePath, sqlContext, fs, fullPartitionPaths).count(),
|
||||
"Must contain " + expTotalRecords + " records");
|
||||
|
||||
// Check that the incremental consumption from prevCommitTime
|
||||
assertEquals(HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(),
|
||||
HoodieClientTestUtils.countRecordsSince(jsc, basePath, sqlContext, timeline, prevCommitTime),
|
||||
"Incremental consumption from " + prevCommitTime + " should give no records in latest commit,"
|
||||
+ " since it is a delete operation");
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Cleaner state corresponding to a partition path.
|
||||
*
|
||||
* @param hoodieCleanStatsTwo List of Clean Stats
|
||||
* @param partitionPath Partition path for filtering
|
||||
* @return Cleaner state corresponding to partition path
|
||||
*/
|
||||
public HoodieCleanStat getCleanStat(List<HoodieCleanStat> hoodieCleanStatsTwo, String partitionPath) {
|
||||
return hoodieCleanStatsTwo.stream().filter(e -> e.getPartitionPath().equals(partitionPath)).findFirst().orElse(null);
|
||||
}
|
||||
|
||||
// Functional Interfaces for passing lambda and Hoodie Write API contexts
|
||||
|
||||
@FunctionalInterface
|
||||
public interface Function2<R, T1, T2> {
|
||||
|
||||
R apply(T1 v1, T2 v2) throws IOException;
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
public interface Function3<R, T1, T2, T3> {
|
||||
|
||||
R apply(T1 v1, T2 v2, T3 v3) throws IOException;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,411 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.testutils.minicluster.HdfsTestService;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocalFileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hudi.table.WorkloadStat;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.TestInfo;
|
||||
import scala.Tuple2;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
|
||||
/**
|
||||
* The test harness for resource initialization and cleanup.
|
||||
*/
|
||||
public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieClientTestHarness.class);
|
||||
|
||||
private String testMethodName;
|
||||
protected transient JavaSparkContext jsc = null;
|
||||
protected transient HoodieSparkEngineContext context = null;
|
||||
protected transient Configuration hadoopConf = null;
|
||||
protected transient SQLContext sqlContext;
|
||||
protected transient FileSystem fs;
|
||||
protected transient HoodieTestDataGenerator dataGen = null;
|
||||
protected transient ExecutorService executorService;
|
||||
protected transient HoodieTableMetaClient metaClient;
|
||||
protected transient SparkRDDWriteClient writeClient;
|
||||
protected transient HoodieReadClient readClient;
|
||||
protected transient HoodieTableFileSystemView tableView;
|
||||
|
||||
protected final SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
|
||||
// dfs
|
||||
protected String dfsBasePath;
|
||||
protected transient HdfsTestService hdfsTestService;
|
||||
protected transient MiniDFSCluster dfsCluster;
|
||||
protected transient DistributedFileSystem dfs;
|
||||
|
||||
@BeforeEach
|
||||
public void setTestMethodName(TestInfo testInfo) {
|
||||
if (testInfo.getTestMethod().isPresent()) {
|
||||
testMethodName = testInfo.getTestMethod().get().getName();
|
||||
} else {
|
||||
testMethodName = "Unknown";
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes resource group for the subclasses of {@link HoodieClientTestBase}.
|
||||
*/
|
||||
public void initResources() throws IOException {
|
||||
initPath();
|
||||
initSparkContexts();
|
||||
initTestDataGenerator();
|
||||
initFileSystem();
|
||||
initMetaClient();
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups resource group for the subclasses of {@link HoodieClientTestBase}.
|
||||
*/
|
||||
public void cleanupResources() throws IOException {
|
||||
cleanupClients();
|
||||
cleanupSparkContexts();
|
||||
cleanupTestDataGenerator();
|
||||
cleanupFileSystem();
|
||||
cleanupDFS();
|
||||
cleanupExecutorService();
|
||||
System.gc();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the Spark contexts ({@link JavaSparkContext} and {@link SQLContext}) with the given application name.
|
||||
*
|
||||
* @param appName The specified application name.
|
||||
*/
|
||||
protected void initSparkContexts(String appName) {
|
||||
// Initialize a local spark env
|
||||
jsc = new JavaSparkContext(HoodieClientTestUtils.getSparkConfForTest(appName + "#" + testMethodName));
|
||||
jsc.setLogLevel("ERROR");
|
||||
hadoopConf = jsc.hadoopConfiguration();
|
||||
|
||||
// SQLContext stuff
|
||||
sqlContext = new SQLContext(jsc);
|
||||
context = new HoodieSparkEngineContext(jsc);
|
||||
hadoopConf = context.getHadoopConf().get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes the Spark contexts ({@link JavaSparkContext} and {@link SQLContext})
|
||||
* with a default name matching the name of the class.
|
||||
*/
|
||||
protected void initSparkContexts() {
|
||||
initSparkContexts(this.getClass().getSimpleName());
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups Spark contexts ({@link JavaSparkContext} and {@link SQLContext}).
|
||||
*/
|
||||
protected void cleanupSparkContexts() {
|
||||
if (sqlContext != null) {
|
||||
LOG.info("Clearing sql context cache of spark-session used in previous test-case");
|
||||
sqlContext.clearCache();
|
||||
sqlContext = null;
|
||||
}
|
||||
|
||||
if (jsc != null) {
|
||||
LOG.info("Closing spark context used in previous test-case");
|
||||
jsc.close();
|
||||
jsc.stop();
|
||||
jsc = null;
|
||||
}
|
||||
|
||||
if (context != null) {
|
||||
LOG.info("Closing spark engine context used in previous test-case");
|
||||
context = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes a file system with the hadoop configuration of Spark context.
|
||||
*/
|
||||
protected void initFileSystem() {
|
||||
if (jsc == null) {
|
||||
throw new IllegalStateException("The Spark context has not been initialized.");
|
||||
}
|
||||
|
||||
initFileSystemWithConfiguration(hadoopConf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes file system with a default empty configuration.
|
||||
*/
|
||||
protected void initFileSystemWithDefaultConfiguration() {
|
||||
initFileSystemWithConfiguration(new Configuration());
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups file system.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void cleanupFileSystem() throws IOException {
|
||||
if (fs != null) {
|
||||
LOG.warn("Closing file-system instance used in previous test-run");
|
||||
fs.close();
|
||||
fs = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by
|
||||
* {@code getTableType()}.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void initMetaClient() throws IOException {
|
||||
if (basePath == null) {
|
||||
throw new IllegalStateException("The base path has not been initialized.");
|
||||
}
|
||||
|
||||
if (jsc == null) {
|
||||
throw new IllegalStateException("The Spark context has not been initialized.");
|
||||
}
|
||||
|
||||
metaClient = HoodieTestUtils.init(context.getHadoopConf().get(), basePath, getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups hoodie clients.
|
||||
*/
|
||||
protected void cleanupClients() throws IOException {
|
||||
if (metaClient != null) {
|
||||
metaClient = null;
|
||||
}
|
||||
if (readClient != null) {
|
||||
readClient = null;
|
||||
}
|
||||
if (writeClient != null) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
if (tableView != null) {
|
||||
tableView.close();
|
||||
tableView = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes a test data generator which used to generate test datas.
|
||||
*
|
||||
*/
|
||||
protected void initTestDataGenerator() {
|
||||
dataGen = new HoodieTestDataGenerator();
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups test data generator.
|
||||
*
|
||||
*/
|
||||
protected void cleanupTestDataGenerator() {
|
||||
if (dataGen != null) {
|
||||
dataGen = null;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes a distributed file system and base directory.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void initDFS() throws IOException {
|
||||
FileSystem.closeAll();
|
||||
hdfsTestService = new HdfsTestService();
|
||||
dfsCluster = hdfsTestService.start(true);
|
||||
|
||||
// Create a temp folder as the base path
|
||||
dfs = dfsCluster.getFileSystem();
|
||||
dfsBasePath = dfs.getWorkingDirectory().toString();
|
||||
this.basePath = dfsBasePath;
|
||||
this.hadoopConf = dfs.getConf();
|
||||
dfs.mkdirs(new Path(dfsBasePath));
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes an instance of {@link HoodieTableMetaClient} with a special table type specified by
|
||||
* {@code getTableType()}.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void initDFSMetaClient() throws IOException {
|
||||
if (dfsBasePath == null) {
|
||||
throw new IllegalStateException("The base path has not been initialized.");
|
||||
}
|
||||
|
||||
if (jsc == null) {
|
||||
throw new IllegalStateException("The Spark context has not been initialized.");
|
||||
}
|
||||
metaClient = HoodieTestUtils.init(dfs.getConf(), dfsBasePath, getTableType());
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups the distributed file system.
|
||||
*
|
||||
* @throws IOException
|
||||
*/
|
||||
protected void cleanupDFS() throws IOException {
|
||||
if (hdfsTestService != null) {
|
||||
hdfsTestService.stop();
|
||||
dfsCluster.shutdown();
|
||||
hdfsTestService = null;
|
||||
dfsCluster = null;
|
||||
dfs = null;
|
||||
}
|
||||
// Need to closeAll to clear FileSystem.Cache, required because DFS and LocalFS used in the
|
||||
// same JVM
|
||||
FileSystem.closeAll();
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes executor service with a fixed thread pool.
|
||||
*
|
||||
* @param threadNum specify the capacity of the fixed thread pool
|
||||
*/
|
||||
protected void initExecutorServiceWithFixedThreadPool(int threadNum) {
|
||||
executorService = Executors.newFixedThreadPool(threadNum);
|
||||
}
|
||||
|
||||
/**
|
||||
* Cleanups the executor service.
|
||||
*/
|
||||
protected void cleanupExecutorService() {
|
||||
if (this.executorService != null) {
|
||||
this.executorService.shutdownNow();
|
||||
this.executorService = null;
|
||||
}
|
||||
}
|
||||
|
||||
private void initFileSystemWithConfiguration(Configuration configuration) {
|
||||
if (basePath == null) {
|
||||
throw new IllegalStateException("The base path has not been initialized.");
|
||||
}
|
||||
|
||||
fs = FSUtils.getFs(basePath, configuration);
|
||||
if (fs instanceof LocalFileSystem) {
|
||||
LocalFileSystem lfs = (LocalFileSystem) fs;
|
||||
// With LocalFileSystem, with checksum disabled, fs.open() returns an inputStream which is FSInputStream
|
||||
// This causes ClassCastExceptions in LogRecordScanner (and potentially other places) calling fs.open
|
||||
// So, for the tests, we enforce checksum verification to circumvent the problem
|
||||
lfs.setVerifyChecksum(true);
|
||||
}
|
||||
}
|
||||
|
||||
public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg) {
|
||||
return getHoodieWriteClient(cfg, false);
|
||||
}
|
||||
|
||||
public HoodieReadClient getHoodieReadClient(String basePath) {
|
||||
readClient = new HoodieReadClient(context, basePath, SQLContext.getOrCreate(jsc.sc()));
|
||||
return readClient;
|
||||
}
|
||||
|
||||
public SparkRDDWriteClient getHoodieWriteClient(HoodieWriteConfig cfg, boolean rollbackInflightCommit) {
|
||||
if (null != writeClient) {
|
||||
writeClient.close();
|
||||
writeClient = null;
|
||||
}
|
||||
writeClient = new SparkRDDWriteClient(context, cfg, rollbackInflightCommit);
|
||||
return writeClient;
|
||||
}
|
||||
|
||||
public HoodieTableMetaClient getHoodieMetaClient(Configuration conf, String basePath) {
|
||||
metaClient = new HoodieTableMetaClient(conf, basePath);
|
||||
return metaClient;
|
||||
}
|
||||
|
||||
public HoodieTableFileSystemView getHoodieTableFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline,
|
||||
FileStatus[] fileStatuses) {
|
||||
if (tableView == null) {
|
||||
tableView = new HoodieTableFileSystemView(metaClient, visibleActiveTimeline, fileStatuses);
|
||||
} else {
|
||||
tableView.init(metaClient, visibleActiveTimeline, fileStatuses);
|
||||
}
|
||||
return tableView;
|
||||
}
|
||||
|
||||
protected Pair<HashMap<String, WorkloadStat>, WorkloadStat> buildProfile(JavaRDD<HoodieRecord> inputRecordsRDD) {
|
||||
HashMap<String, WorkloadStat> partitionPathStatMap = new HashMap<>();
|
||||
WorkloadStat globalStat = new WorkloadStat();
|
||||
|
||||
// group the records by partitionPath + currentLocation combination, count the number of
|
||||
// records in each partition
|
||||
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = inputRecordsRDD
|
||||
.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
|
||||
.countByKey();
|
||||
|
||||
// count the number of both inserts and updates in each partition, update the counts to workLoadStats
|
||||
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
|
||||
String partitionPath = e.getKey()._1();
|
||||
Long count = e.getValue();
|
||||
Option<HoodieRecordLocation> locOption = e.getKey()._2();
|
||||
|
||||
if (!partitionPathStatMap.containsKey(partitionPath)) {
|
||||
partitionPathStatMap.put(partitionPath, new WorkloadStat());
|
||||
}
|
||||
|
||||
if (locOption.isPresent()) {
|
||||
// update
|
||||
partitionPathStatMap.get(partitionPath).addUpdates(locOption.get(), count);
|
||||
globalStat.addUpdates(locOption.get(), count);
|
||||
} else {
|
||||
// insert
|
||||
partitionPathStatMap.get(partitionPath).addInserts(count);
|
||||
globalStat.addInserts(count);
|
||||
}
|
||||
}
|
||||
return Pair.of(partitionPathStatMap, globalStat);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,203 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.client.HoodieReadClient;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
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.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Cell;
|
||||
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
/**
|
||||
* Utility methods to aid testing inside the HoodieClient module.
|
||||
*/
|
||||
public class HoodieClientTestUtils {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieClientTestUtils.class);
|
||||
|
||||
/**
|
||||
* Returns a Spark config for this test.
|
||||
*
|
||||
* The following properties may be set to customize the Spark context:
|
||||
* SPARK_EVLOG_DIR: Local directory where event logs should be saved. This
|
||||
* allows viewing the logs with spark-history-server.
|
||||
*
|
||||
* @note When running the tests using maven, use the following syntax to set
|
||||
* a property:
|
||||
* mvn -DSPARK_XXX=yyy ...
|
||||
*
|
||||
* @param appName A name for the Spark application. Shown in the Spark web UI.
|
||||
* @return A Spark config
|
||||
*/
|
||||
public static SparkConf getSparkConfForTest(String appName) {
|
||||
SparkConf sparkConf = new SparkConf().setAppName(appName)
|
||||
.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer").setMaster("local[8]");
|
||||
|
||||
String evlogDir = System.getProperty("SPARK_EVLOG_DIR");
|
||||
if (evlogDir != null) {
|
||||
sparkConf.set("spark.eventLog.enabled", "true");
|
||||
sparkConf.set("spark.eventLog.dir", evlogDir);
|
||||
}
|
||||
|
||||
return HoodieReadClient.addHoodieSupport(sparkConf);
|
||||
}
|
||||
|
||||
private static HashMap<String, String> getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline,
|
||||
List<HoodieInstant> commitsToReturn) throws IOException {
|
||||
HashMap<String, String> fileIdToFullPath = new HashMap<>();
|
||||
for (HoodieInstant commit : commitsToReturn) {
|
||||
HoodieCommitMetadata metadata =
|
||||
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class);
|
||||
fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(basePath));
|
||||
}
|
||||
return fileIdToFullPath;
|
||||
}
|
||||
|
||||
public static Dataset<Row> readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline,
|
||||
String instantTime) {
|
||||
HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime);
|
||||
if (!commitTimeline.containsInstant(commitInstant)) {
|
||||
throw new HoodieException("No commit exists at " + instantTime);
|
||||
}
|
||||
try {
|
||||
HashMap<String, String> paths =
|
||||
getLatestFileIDsToFullPath(basePath, commitTimeline, Arrays.asList(commitInstant));
|
||||
LOG.info("Path :" + paths.values());
|
||||
return sqlContext.read().parquet(paths.values().toArray(new String[paths.size()]))
|
||||
.filter(String.format("%s ='%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, instantTime));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading commit " + instantTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain all new data written into the Hoodie table since the given timestamp.
|
||||
*/
|
||||
public static long countRecordsSince(JavaSparkContext jsc, String basePath, SQLContext sqlContext,
|
||||
HoodieTimeline commitTimeline, String lastCommitTime) {
|
||||
List<HoodieInstant> commitsToReturn =
|
||||
commitTimeline.findInstantsAfter(lastCommitTime, Integer.MAX_VALUE).getInstants().collect(Collectors.toList());
|
||||
try {
|
||||
// Go over the commit metadata, and obtain the new files that need to be read.
|
||||
HashMap<String, String> fileIdToFullPath = getLatestFileIDsToFullPath(basePath, commitTimeline, commitsToReturn);
|
||||
String[] paths = fileIdToFullPath.values().toArray(new String[fileIdToFullPath.size()]);
|
||||
if (paths[0].endsWith(HoodieFileFormat.PARQUET.getFileExtension())) {
|
||||
return sqlContext.read().parquet(paths)
|
||||
.filter(String.format("%s >'%s'", HoodieRecord.COMMIT_TIME_METADATA_FIELD, lastCommitTime))
|
||||
.count();
|
||||
} else if (paths[0].endsWith(HoodieFileFormat.HFILE.getFileExtension())) {
|
||||
return readHFile(jsc, paths)
|
||||
.filter(gr -> HoodieTimeline.compareTimestamps(lastCommitTime, HoodieActiveTimeline.LESSER_THAN,
|
||||
gr.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()))
|
||||
.count();
|
||||
}
|
||||
throw new HoodieException("Unsupported base file format for file :" + paths[0]);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error pulling data incrementally from commitTimestamp :" + lastCommitTime, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Reads the paths under the a hoodie table out as a DataFrame.
|
||||
*/
|
||||
public static Dataset<Row> read(JavaSparkContext jsc, String basePath, SQLContext sqlContext, FileSystem fs,
|
||||
String... paths) {
|
||||
List<String> filteredPaths = new ArrayList<>();
|
||||
try {
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), basePath, true);
|
||||
for (String path : paths) {
|
||||
BaseFileOnlyView fileSystemView = new HoodieTableFileSystemView(metaClient,
|
||||
metaClient.getCommitsTimeline().filterCompletedInstants(), fs.globStatus(new Path(path)));
|
||||
List<HoodieBaseFile> latestFiles = fileSystemView.getLatestBaseFiles().collect(Collectors.toList());
|
||||
for (HoodieBaseFile file : latestFiles) {
|
||||
filteredPaths.add(file.getPath());
|
||||
}
|
||||
}
|
||||
return sqlContext.read().parquet(filteredPaths.toArray(new String[filteredPaths.size()]));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error reading hoodie table as a dataframe", e);
|
||||
}
|
||||
}
|
||||
|
||||
public static Stream<GenericRecord> readHFile(JavaSparkContext jsc, String[] paths) {
|
||||
// TODO: this should be ported to use HoodieStorageReader
|
||||
List<GenericRecord> valuesAsList = new LinkedList<>();
|
||||
|
||||
FileSystem fs = FSUtils.getFs(paths[0], jsc.hadoopConfiguration());
|
||||
CacheConfig cacheConfig = new CacheConfig(fs.getConf());
|
||||
Schema schema = null;
|
||||
for (String path : paths) {
|
||||
try {
|
||||
HFile.Reader reader = HFile.createReader(fs, new Path(path), cacheConfig, fs.getConf());
|
||||
if (schema == null) {
|
||||
schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get("schema".getBytes())));
|
||||
}
|
||||
HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (!scanner.seekTo()) {
|
||||
// EOF reached
|
||||
continue;
|
||||
}
|
||||
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
byte[] value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
valuesAsList.add(HoodieAvroUtils.bytesToAvro(value, schema));
|
||||
} while (scanner.next());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
|
||||
}
|
||||
}
|
||||
return valuesAsList.stream();
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,165 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.GenericRecordBuilder;
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
|
||||
import org.apache.hadoop.hive.ql.io.IOConstants;
|
||||
import org.apache.hadoop.hive.serde2.ColumnProjectionUtils;
|
||||
import org.apache.hadoop.io.ArrayWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.mapred.FileInputFormat;
|
||||
import org.apache.hadoop.mapred.InputSplit;
|
||||
import org.apache.hadoop.mapred.JobConf;
|
||||
import org.apache.hadoop.mapred.RecordReader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Utility methods to aid in testing MergeOnRead (workaround for HoodieReadClient for MOR).
|
||||
*/
|
||||
public class HoodieMergeOnReadTestUtils {
|
||||
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
||||
String basePath) {
|
||||
return getRecordsUsingInputFormat(conf, inputPaths, basePath, new JobConf(conf), true);
|
||||
}
|
||||
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths,
|
||||
String basePath, JobConf jobConf, boolean realtime) {
|
||||
Schema schema = new Schema.Parser().parse(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA);
|
||||
return getRecordsUsingInputFormat(conf, inputPaths, basePath, jobConf, realtime, schema,
|
||||
HoodieTestDataGenerator.TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||
}
|
||||
|
||||
public static List<GenericRecord> getRecordsUsingInputFormat(Configuration conf, List<String> inputPaths, String basePath, JobConf jobConf, boolean realtime, Schema rawSchema,
|
||||
String rawHiveColumnTypes, boolean projectCols, List<String> projectedColumns) {
|
||||
|
||||
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf, basePath);
|
||||
FileInputFormat inputFormat = HoodieInputFormatUtils.getInputFormat(metaClient.getTableConfig().getBaseFileFormat(), realtime, jobConf);
|
||||
|
||||
Schema schema = HoodieAvroUtils.addMetadataFields(rawSchema);
|
||||
String hiveColumnTypes = HoodieAvroUtils.addMetadataColumnTypes(rawHiveColumnTypes);
|
||||
setPropsForInputFormat(inputFormat, jobConf, schema, hiveColumnTypes, projectCols, projectedColumns);
|
||||
final List<Field> fields;
|
||||
if (projectCols) {
|
||||
fields = schema.getFields().stream().filter(f -> projectedColumns.contains(f.name()))
|
||||
.collect(Collectors.toList());
|
||||
} else {
|
||||
fields = schema.getFields();
|
||||
}
|
||||
final Schema projectedSchema = Schema.createRecord(fields.stream()
|
||||
.map(f -> new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal()))
|
||||
.collect(Collectors.toList()));
|
||||
|
||||
return inputPaths.stream().map(path -> {
|
||||
setInputPath(jobConf, path);
|
||||
List<GenericRecord> records = new ArrayList<>();
|
||||
try {
|
||||
List<InputSplit> splits = Arrays.asList(inputFormat.getSplits(jobConf, 1));
|
||||
for (InputSplit split : splits) {
|
||||
RecordReader recordReader = inputFormat.getRecordReader(split, jobConf, null);
|
||||
Object key = recordReader.createKey();
|
||||
ArrayWritable writable = (ArrayWritable) recordReader.createValue();
|
||||
while (recordReader.next(key, writable)) {
|
||||
GenericRecordBuilder newRecord = new GenericRecordBuilder(projectedSchema);
|
||||
// writable returns an array with [field1, field2, _hoodie_commit_time,
|
||||
// _hoodie_commit_seqno]
|
||||
Writable[] values = writable.get();
|
||||
schema.getFields().stream()
|
||||
.filter(f -> !projectCols || projectedColumns.contains(f.name()))
|
||||
.map(f -> Pair.of(projectedSchema.getFields().stream()
|
||||
.filter(p -> f.name().equals(p.name())).findFirst().get(), f))
|
||||
.forEach(fieldsPair -> newRecord.set(fieldsPair.getKey(), values[fieldsPair.getValue().pos()]));
|
||||
records.add(newRecord.build());
|
||||
}
|
||||
}
|
||||
} catch (IOException ie) {
|
||||
ie.printStackTrace();
|
||||
}
|
||||
return records;
|
||||
}).reduce((a, b) -> {
|
||||
a.addAll(b);
|
||||
return a;
|
||||
}).orElse(new ArrayList<>());
|
||||
}
|
||||
|
||||
private static void setPropsForInputFormat(FileInputFormat inputFormat, JobConf jobConf, Schema schema, String hiveColumnTypes, boolean projectCols, List<String> projectedCols) {
|
||||
List<Schema.Field> fields = schema.getFields();
|
||||
final List<String> projectedColNames;
|
||||
if (!projectCols) {
|
||||
projectedColNames = fields.stream().map(Field::name).collect(Collectors.toList());
|
||||
} else {
|
||||
projectedColNames = projectedCols;
|
||||
}
|
||||
|
||||
String names = fields.stream()
|
||||
.filter(f -> projectedColNames.contains(f.name()))
|
||||
.map(f -> f.name()).collect(Collectors.joining(","));
|
||||
String positions = fields.stream()
|
||||
.filter(f -> projectedColNames.contains(f.name()))
|
||||
.map(f -> String.valueOf(f.pos())).collect(Collectors.joining(","));
|
||||
String hiveColumnNames = fields.stream()
|
||||
.filter(field -> !field.name().equalsIgnoreCase("datestr"))
|
||||
.map(Schema.Field::name).collect(Collectors.joining(","));
|
||||
hiveColumnNames = hiveColumnNames + ",datestr";
|
||||
|
||||
Configuration conf = HoodieTestUtils.getDefaultHadoopConf();
|
||||
String hiveColumnTypesWithDatestr = hiveColumnTypes + ",string";
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypesWithDatestr);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
jobConf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
jobConf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
|
||||
conf.set(hive_metastoreConstants.META_TABLE_COLUMNS, hiveColumnNames);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR, names);
|
||||
conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR, positions);
|
||||
conf.set(hive_metastoreConstants.META_TABLE_PARTITION_COLUMNS, "datestr");
|
||||
conf.set(hive_metastoreConstants.META_TABLE_COLUMN_TYPES, hiveColumnTypesWithDatestr);
|
||||
conf.set(IOConstants.COLUMNS, hiveColumnNames);
|
||||
conf.get(IOConstants.COLUMNS_TYPES, hiveColumnTypesWithDatestr);
|
||||
|
||||
// Hoodie Input formats are also configurable
|
||||
Configurable configurable = (Configurable)inputFormat;
|
||||
configurable.setConf(conf);
|
||||
jobConf.addResource(conf);
|
||||
}
|
||||
|
||||
private static void setInputPath(JobConf jobConf, String inputPath) {
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||
jobConf.set("mapreduce.input.fileinputformat.inputdir", inputPath);
|
||||
jobConf.set("map.input.dir", inputPath);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.FileCreateUtils;
|
||||
import org.apache.hudi.common.testutils.HoodieTestTable;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieParquetWriter;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroSchemaConverter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
|
||||
|
||||
import java.nio.file.Paths;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName;
|
||||
|
||||
public class HoodieWriteableTestTable extends HoodieTestTable {
|
||||
|
||||
private final Schema schema;
|
||||
private final BloomFilter filter;
|
||||
|
||||
private HoodieWriteableTestTable(String basePath, FileSystem fs, HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
|
||||
super(basePath, fs, metaClient);
|
||||
this.schema = schema;
|
||||
this.filter = filter;
|
||||
}
|
||||
|
||||
public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema, BloomFilter filter) {
|
||||
return new HoodieWriteableTestTable(metaClient.getBasePath(), metaClient.getRawFs(), metaClient, schema, filter);
|
||||
}
|
||||
|
||||
public static HoodieWriteableTestTable of(HoodieTableMetaClient metaClient, Schema schema) {
|
||||
BloomFilter filter = BloomFilterFactory
|
||||
.createBloomFilter(10000, 0.0000001, -1, BloomFilterTypeCode.SIMPLE.name());
|
||||
return of(metaClient, schema, filter);
|
||||
}
|
||||
|
||||
public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
return of(metaClient, schema);
|
||||
}
|
||||
|
||||
public static HoodieWriteableTestTable of(HoodieTable hoodieTable, Schema schema, BloomFilter filter) {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
return of(metaClient, schema, filter);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteableTestTable addCommit(String instantTime) throws Exception {
|
||||
return (HoodieWriteableTestTable) super.addCommit(instantTime);
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteableTestTable forCommit(String instantTime) {
|
||||
return (HoodieWriteableTestTable) super.forCommit(instantTime);
|
||||
}
|
||||
|
||||
public String withInserts(String partition) throws Exception {
|
||||
return withInserts(partition, new HoodieRecord[0]);
|
||||
}
|
||||
|
||||
public String withInserts(String partition, HoodieRecord... records) throws Exception {
|
||||
String fileId = UUID.randomUUID().toString();
|
||||
withInserts(partition, fileId, records);
|
||||
return fileId;
|
||||
}
|
||||
|
||||
public HoodieWriteableTestTable withInserts(String partition, String fileId) throws Exception {
|
||||
return withInserts(partition, fileId, new HoodieRecord[0]);
|
||||
}
|
||||
|
||||
public HoodieWriteableTestTable withInserts(String partition, String fileId, HoodieRecord... records) throws Exception {
|
||||
FileCreateUtils.createPartitionMetaFile(basePath, partition);
|
||||
String fileName = baseFileName(currentInstantTime, fileId);
|
||||
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, filter);
|
||||
HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
new Configuration(), Double.parseDouble(HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO));
|
||||
try (HoodieParquetWriter writer = new HoodieParquetWriter(
|
||||
currentInstantTime,
|
||||
new Path(Paths.get(basePath, partition, fileName).toString()),
|
||||
config, schema, new SparkTaskContextSupplier())) {
|
||||
int seqId = 1;
|
||||
for (HoodieRecord record : records) {
|
||||
GenericRecord avroRecord = (GenericRecord) record.getData().getInsertValue(schema).get();
|
||||
HoodieAvroUtils.addCommitMetadataToRecord(avroRecord, currentInstantTime, String.valueOf(seqId++));
|
||||
HoodieAvroUtils.addHoodieKeyToRecord(avroRecord, record.getRecordKey(), record.getPartitionPath(), fileName);
|
||||
writer.writeAvro(record.getRecordKey(), avroRecord);
|
||||
filter.add(record.getRecordKey());
|
||||
}
|
||||
}
|
||||
|
||||
return this;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,85 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
/**
|
||||
* A custom {@link WriteStatus} that merges passed metadata key value map to {@code WriteStatus.markSuccess()} and
|
||||
* {@code WriteStatus.markFailure()}.
|
||||
*/
|
||||
public class MetadataMergeWriteStatus extends WriteStatus {
|
||||
|
||||
private Map<String, String> mergedMetadataMap = new HashMap<>();
|
||||
|
||||
public MetadataMergeWriteStatus(Boolean trackSuccessRecords, Double failureFraction) {
|
||||
super(trackSuccessRecords, failureFraction);
|
||||
}
|
||||
|
||||
public static Map<String, String> mergeMetadataForWriteStatuses(List<WriteStatus> writeStatuses) {
|
||||
Map<String, String> allWriteStatusMergedMetadataMap = new HashMap<>();
|
||||
for (WriteStatus writeStatus : writeStatuses) {
|
||||
MetadataMergeWriteStatus.mergeMetadataMaps(((MetadataMergeWriteStatus) writeStatus).getMergedMetadataMap(),
|
||||
allWriteStatusMergedMetadataMap);
|
||||
}
|
||||
return allWriteStatusMergedMetadataMap;
|
||||
}
|
||||
|
||||
private static void mergeMetadataMaps(Map<String, String> mergeFromMap, Map<String, String> mergeToMap) {
|
||||
for (Entry<String, String> entry : mergeFromMap.entrySet()) {
|
||||
String key = entry.getKey();
|
||||
if (!mergeToMap.containsKey(key)) {
|
||||
mergeToMap.put(key, "0");
|
||||
}
|
||||
mergeToMap.put(key, addStrsAsInt(entry.getValue(), mergeToMap.get(key)));
|
||||
}
|
||||
}
|
||||
|
||||
private static String addStrsAsInt(String a, String b) {
|
||||
return String.valueOf(Integer.parseInt(a) + Integer.parseInt(b));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markSuccess(HoodieRecord record, Option<Map<String, String>> recordMetadata) {
|
||||
super.markSuccess(record, recordMetadata);
|
||||
if (recordMetadata.isPresent()) {
|
||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> recordMetadata) {
|
||||
super.markFailure(record, t, recordMetadata);
|
||||
if (recordMetadata.isPresent()) {
|
||||
mergeMetadataMaps(recordMetadata.get(), mergedMetadataMap);
|
||||
}
|
||||
}
|
||||
|
||||
private Map<String, String> getMergedMetadataMap() {
|
||||
return mergedMetadataMap;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,176 @@
|
||||
/*
|
||||
* 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.testutils;
|
||||
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
|
||||
import org.apache.spark.sql.Dataset;
|
||||
import org.apache.spark.sql.Row;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer$;
|
||||
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder;
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder;
|
||||
import org.apache.spark.sql.catalyst.expressions.Attribute;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
|
||||
import org.apache.spark.sql.catalyst.expressions.GenericRow;
|
||||
import org.apache.spark.sql.types.DataTypes;
|
||||
import org.apache.spark.sql.types.Metadata;
|
||||
import org.apache.spark.sql.types.StructField;
|
||||
import org.apache.spark.sql.types.StructType;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import scala.collection.JavaConversions;
|
||||
import scala.collection.JavaConverters;
|
||||
|
||||
import static org.apache.hudi.common.testutils.FileSystemTestUtils.RANDOM;
|
||||
|
||||
/**
|
||||
* Dataset test utils.
|
||||
*/
|
||||
public class SparkDatasetTestUtils {
|
||||
|
||||
public static final StructType STRUCT_TYPE = new StructType(new StructField[] {
|
||||
new StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomLong", DataTypes.LongType, false, Metadata.empty())});
|
||||
|
||||
public static final StructType ERROR_STRUCT_TYPE = new StructType(new StructField[] {
|
||||
new StructField(HoodieRecord.COMMIT_TIME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.COMMIT_SEQNO_METADATA_FIELD, DataTypes.LongType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.RECORD_KEY_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.PARTITION_PATH_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField(HoodieRecord.FILENAME_METADATA_FIELD, DataTypes.StringType, false, Metadata.empty()),
|
||||
new StructField("randomInt", DataTypes.IntegerType, false, Metadata.empty()),
|
||||
new StructField("randomStr", DataTypes.StringType, false, Metadata.empty())});
|
||||
|
||||
public static final ExpressionEncoder ENCODER = getEncoder(STRUCT_TYPE);
|
||||
public static final ExpressionEncoder ERROR_ENCODER = getEncoder(ERROR_STRUCT_TYPE);
|
||||
|
||||
/**
|
||||
* Generate Encode for the passed in {@link StructType}.
|
||||
*
|
||||
* @param schema instance of {@link StructType} for which encoder is requested.
|
||||
* @return the encoder thus generated.
|
||||
*/
|
||||
private static ExpressionEncoder getEncoder(StructType schema) {
|
||||
List<Attribute> attributes = JavaConversions.asJavaCollection(schema.toAttributes()).stream()
|
||||
.map(Attribute::toAttribute).collect(Collectors.toList());
|
||||
return RowEncoder.apply(schema)
|
||||
.resolveAndBind(JavaConverters.asScalaBufferConverter(attributes).asScala().toSeq(),
|
||||
SimpleAnalyzer$.MODULE$);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate random Rows.
|
||||
*
|
||||
* @param count total number of Rows to be generated.
|
||||
* @param partitionPath partition path to be set
|
||||
* @return the Dataset<Row>s thus generated.
|
||||
*/
|
||||
public static Dataset<Row> getRandomRows(SQLContext sqlContext, int count, String partitionPath, boolean isError) {
|
||||
List<Row> records = new ArrayList<>();
|
||||
for (long recordNum = 0; recordNum < count; recordNum++) {
|
||||
records.add(getRandomValue(partitionPath, isError));
|
||||
}
|
||||
return sqlContext.createDataFrame(records, isError ? ERROR_STRUCT_TYPE : STRUCT_TYPE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Generate random Row.
|
||||
*
|
||||
* @param partitionPath partition path to be set in the Row.
|
||||
* @return the Row thus generated.
|
||||
*/
|
||||
public static Row getRandomValue(String partitionPath, boolean isError) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
Object[] values = new Object[7];
|
||||
values[0] = ""; //commit time
|
||||
if (!isError) {
|
||||
values[1] = ""; // commit seq no
|
||||
} else {
|
||||
values[1] = RANDOM.nextLong();
|
||||
}
|
||||
values[2] = UUID.randomUUID().toString();
|
||||
values[3] = partitionPath;
|
||||
values[4] = ""; // filename
|
||||
values[5] = RANDOM.nextInt();
|
||||
if (!isError) {
|
||||
values[6] = RANDOM.nextLong();
|
||||
} else {
|
||||
values[6] = UUID.randomUUID().toString();
|
||||
}
|
||||
return new GenericRow(values);
|
||||
}
|
||||
|
||||
/**
|
||||
* Convert Dataset<Row>s to List of {@link InternalRow}s.
|
||||
*
|
||||
* @param rows Dataset<Row>s to be converted
|
||||
* @return the List of {@link InternalRow}s thus converted.
|
||||
*/
|
||||
public static List<InternalRow> toInternalRows(Dataset<Row> rows, ExpressionEncoder encoder) {
|
||||
List<InternalRow> toReturn = new ArrayList<>();
|
||||
List<Row> rowList = rows.collectAsList();
|
||||
for (Row row : rowList) {
|
||||
toReturn.add(encoder.toRow(row).copy());
|
||||
}
|
||||
return toReturn;
|
||||
}
|
||||
|
||||
public static InternalRow getInternalRowWithError(String partitionPath) {
|
||||
// order commit time, seq no, record key, partition path, file name
|
||||
String recordKey = UUID.randomUUID().toString();
|
||||
Object[] values = new Object[7];
|
||||
values[0] = "";
|
||||
values[1] = "";
|
||||
values[2] = recordKey;
|
||||
values[3] = partitionPath;
|
||||
values[4] = "";
|
||||
values[5] = RANDOM.nextInt();
|
||||
values[6] = RANDOM.nextBoolean();
|
||||
return new GenericInternalRow(values);
|
||||
}
|
||||
|
||||
public static HoodieWriteConfig.Builder getConfigBuilder(String basePath) {
|
||||
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
|
||||
.withParallelism(2, 2)
|
||||
.withDeleteParallelism(2)
|
||||
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024).build())
|
||||
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024).parquetMaxFileSize(1024 * 1024).build())
|
||||
.forTable("test-trip-table")
|
||||
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
|
||||
.withBulkInsertParallelism(2);
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* 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.testutils.providers;
|
||||
|
||||
import org.apache.spark.SparkConf;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.sql.SQLContext;
|
||||
import org.apache.spark.sql.SparkSession;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.Map;
|
||||
|
||||
public interface SparkProvider extends org.apache.hudi.testutils.providers.HoodieEngineContextProvider {
|
||||
|
||||
SparkSession spark();
|
||||
|
||||
SQLContext sqlContext();
|
||||
|
||||
JavaSparkContext jsc();
|
||||
|
||||
default SparkConf conf(Map<String, String> overwritingConfigs) {
|
||||
SparkConf sparkConf = new SparkConf();
|
||||
sparkConf.set("spark.app.name", getClass().getName());
|
||||
sparkConf.set("spark.master", "local[*]");
|
||||
sparkConf.set("spark.driver.maxResultSize", "2g");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compress", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "true");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.codec", "org.apache.hadoop.io.compress.GzipCodec");
|
||||
sparkConf.set("spark.hadoop.mapred.output.compression.type", "BLOCK");
|
||||
sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer");
|
||||
overwritingConfigs.forEach(sparkConf::set);
|
||||
return sparkConf;
|
||||
}
|
||||
|
||||
default SparkConf conf() {
|
||||
return conf(Collections.emptyMap());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
|
||||
# CONSOLE is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# CONSOLE uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,31 @@
|
||||
###
|
||||
# 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.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache=INFO
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
log4j.logger.org.apache.hadoop.hbase=ERROR
|
||||
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* 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.
|
||||
*/
|
||||
{
|
||||
"type" : "record",
|
||||
"name" : "triprec",
|
||||
"fields" : [
|
||||
{
|
||||
"name" : "timestamp",
|
||||
"type" : "long"
|
||||
}, {
|
||||
"name" : "_row_key",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "rider",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "driver",
|
||||
"type" : "string"
|
||||
}, {
|
||||
"name" : "begin_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "begin_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lat",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "end_lon",
|
||||
"type" : "double"
|
||||
}, {
|
||||
"name" : "distance_in_meters",
|
||||
"type" : "int"
|
||||
}, {
|
||||
"name" : "seconds_since_epoch",
|
||||
"type" : "long"
|
||||
}, {
|
||||
"name" : "weight",
|
||||
"type" : "float"
|
||||
},{
|
||||
"name" : "nation",
|
||||
"type" : "bytes"
|
||||
},{
|
||||
"name" : "current_date",
|
||||
"type" : {
|
||||
"type" : "int",
|
||||
"logicalType" : "date"
|
||||
}
|
||||
},{
|
||||
"name" : "current_ts",
|
||||
"type" : {
|
||||
"type" : "long"
|
||||
}
|
||||
},{
|
||||
"name" : "height",
|
||||
"type" : {
|
||||
"type" : "fixed",
|
||||
"name" : "abc",
|
||||
"size" : 5,
|
||||
"logicalType" : "decimal",
|
||||
"precision" : 10,
|
||||
"scale": 6
|
||||
}
|
||||
}, {
|
||||
"name" :"city_to_state",
|
||||
"type" : {
|
||||
"type" : "map",
|
||||
"values": "string"
|
||||
}
|
||||
},
|
||||
{
|
||||
"name" : "fare",
|
||||
"type" : {
|
||||
"type" : "record",
|
||||
"name" : "fare",
|
||||
"fields" : [
|
||||
{
|
||||
"name" : "amount",
|
||||
"type" : "double"
|
||||
},
|
||||
{
|
||||
"name" : "currency",
|
||||
"type" : "string"
|
||||
}
|
||||
]
|
||||
}
|
||||
},
|
||||
{
|
||||
"name" : "tip_history",
|
||||
"type" : {
|
||||
"type" : "array",
|
||||
"items" : {
|
||||
"type" : "record",
|
||||
"name" : "tip_history",
|
||||
"fields" : [
|
||||
{
|
||||
"name" : "amount",
|
||||
"type" : "double"
|
||||
},
|
||||
{
|
||||
"name" : "currency",
|
||||
"type" : "string"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
},
|
||||
{
|
||||
"name" : "_hoodie_is_deleted",
|
||||
"type" : "boolean",
|
||||
"default" : false
|
||||
} ]
|
||||
}
|
||||
Reference in New Issue
Block a user